delete_merge_off_heap

This commit is contained in:
yanhuqing
2017-11-23 14:56:56 +08:00
parent d83edd05cc
commit 3bb0a2f533
100 changed files with 14 additions and 15642 deletions

View File

@@ -176,11 +176,6 @@
<Bug category="MT_CORRECTNESS"/>
<Class name="com.actiontech.dble.server.handler.ServerLoadDataInfileHandler"/>
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
<Match>
<Bug category="MT_CORRECTNESS"/>
<Class name="com.actiontech.dble.memory.unsafe.map.BytesToBytesMap$MapIterator"/>
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
<Match>
<Bug category="MT_CORRECTNESS"/>

View File

@@ -26,7 +26,6 @@ import com.actiontech.dble.config.util.ConfigUtil;
import com.actiontech.dble.config.util.DnPropertyUtil;
import com.actiontech.dble.log.transaction.TxnLogProcessor;
import com.actiontech.dble.manager.ManagerConnectionFactory;
import com.actiontech.dble.memory.SeverMemory;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.meta.ProxyMetaManager;
import com.actiontech.dble.net.*;
@@ -95,12 +94,6 @@ public final class DbleServer {
private final AtomicLong xaIDInc = new AtomicLong();
/**
* Memory Manager
*/
private SeverMemory serverMemory = null;
private final ReentrantReadWriteLock confLock = new ReentrantReadWriteLock();
private final ReentrantReadWriteLock metaLock = new ReentrantReadWriteLock();
@@ -217,10 +210,6 @@ public final class DbleServer {
}
}
public SeverMemory getServerMemory() {
return serverMemory;
}
public XASessionCheck getXaSessionCheck() {
return xaSessionCheck;
}
@@ -333,17 +322,6 @@ public final class DbleServer {
}
bufferPool = new DirectByteBufferPool(bufferPoolPageSize, bufferPoolChunkSize, bufferPoolPageNumber);
// Off Heap For Merge/Order/Group/Limit
if (system.getUseOffHeapForMerge() == 1) {
try {
serverMemory = new SeverMemory(system, totalNetWorkBufferSize);
} catch (NoSuchFieldException e) {
LOGGER.error("NoSuchFieldException", e);
} catch (IllegalAccessException e) {
LOGGER.error("Error", e);
}
}
int threadPoolSize = system.getProcessorExecutor();
businessExecutor = ExecutorUtil.createFixed("BusinessExecutor", threadPoolSize);
complexQueryExecutor = ExecutorUtil.createCached("complexQueryExecutor", threadPoolSize);

View File

@@ -114,10 +114,6 @@ public final class SystemConfig {
private int useGlobleTableCheck = 1;
private long glableTableCheckPeriod;
/**
* Off Heap for Merge 1-enable,0-disable
*/
private int useOffHeapForMerge;
/*
* memoryPageSize the unit is M
*/
@@ -171,7 +167,6 @@ public final class SystemConfig {
this.txIsolation = Isolations.REPEATED_READ;
this.sqlRecordCount = DEFAULT_SQL_RECORD_COUNT;
this.glableTableCheckPeriod = DEFAULT_GLOBAL_TABLE_CHECK_PERIOD;
this.useOffHeapForMerge = 1;
this.memoryPageSize = MEMORY_PAGE_SIZE;
this.spillsFileBufferSize = SPILLS_FILE_BUFFER_SIZE;
this.xaRecoveryLogBaseDir = SystemConfig.getHomePath() + "/tmlogs/";
@@ -229,15 +224,6 @@ public final class SystemConfig {
this.transactionLogBaseName = transactionLogBaseName;
}
public int getUseOffHeapForMerge() {
return useOffHeapForMerge;
}
@SuppressWarnings("unused")
public void setUseOffHeapForMerge(int useOffHeapForMerge) {
this.useOffHeapForMerge = useOffHeapForMerge;
}
public String getMemoryPageSize() {
return memoryPageSize;
}

View File

@@ -9,7 +9,6 @@ import com.actiontech.dble.DbleServer;
import com.actiontech.dble.backend.mysql.PacketUtil;
import com.actiontech.dble.config.Fields;
import com.actiontech.dble.manager.ManagerConnection;
import com.actiontech.dble.memory.SeverMemory;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
import com.actiontech.dble.net.mysql.EOFPacket;
@@ -23,11 +22,8 @@ import java.util.Map;
import java.util.concurrent.ConcurrentMap;
/**
* show@@directmemory
* show @@directmemory
*
* @author zagnix
* @version 1.0
* @create 2016-09-21 17:35
*/
public final class ShowDirectMemory {
@@ -40,7 +36,7 @@ public final class ShowDirectMemory {
private static final EOFPacket DETAIL_EOF = new EOFPacket();
private static final int TOTAL_FIELD_COUNT = 5;
private static final int TOTAL_FIELD_COUNT = 3;
private static final ResultSetHeaderPacket TOTAL_HEADER = PacketUtil.getHeader(TOTAL_FIELD_COUNT);
private static final FieldPacket[] TOTAL_FIELDS = new FieldPacket[TOTAL_FIELD_COUNT];
private static final EOFPacket TOTAL_EOF = new EOFPacket();
@@ -57,7 +53,7 @@ public final class ShowDirectMemory {
DETAIL_FIELDS[i++].setPacketId(++packetId);
DETAIL_FIELDS[i] = PacketUtil.getField(" SIZE ", Fields.FIELD_TYPE_VAR_STRING);
DETAIL_FIELDS[i++].setPacketId(++packetId);
DETAIL_FIELDS[i].setPacketId(++packetId);
DETAIL_EOF.setPacketId(++packetId);
@@ -66,20 +62,14 @@ public final class ShowDirectMemory {
TOTAL_HEADER.setPacketId(++packetId);
TOTAL_FIELDS[i] = PacketUtil.getField("MDIRECT_MEMORY_MAXED", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i] = PacketUtil.getField("DIRECT_MEMORY_MAXED", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i++].setPacketId(++packetId);
TOTAL_FIELDS[i] = PacketUtil.getField("DIRECT_MEMORY_USED", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i++].setPacketId(++packetId);
TOTAL_FIELDS[i] = PacketUtil.getField("DIRECT_MEMORY_AVAILABLE", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i++].setPacketId(++packetId);
TOTAL_FIELDS[i] = PacketUtil.getField("SAFETY_FRACTION", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i++].setPacketId(++packetId);
TOTAL_FIELDS[i] = PacketUtil.getField("DIRECT_MEMORY_RESERVED", Fields.FIELD_TYPE_VAR_STRING);
TOTAL_FIELDS[i++].setPacketId(++packetId);
TOTAL_FIELDS[i].setPacketId(++packetId);
TOTAL_EOF.setPacketId(++packetId);
@@ -96,7 +86,7 @@ public final class ShowDirectMemory {
}
public static void showDirectMemoryDetail(ManagerConnection c) {
private static void showDirectMemoryDetail(ManagerConnection c) {
ByteBuffer buffer = c.allocate();
@@ -114,36 +104,14 @@ public final class ShowDirectMemory {
// write rows
byte packetId = DETAIL_EOF.getPacketId();
int useOffHeapForMerge = DbleServer.getInstance().getConfig().getSystem().getUseOffHeapForMerge();
ConcurrentMap<Long, Long> networkBufferPool = DbleServer.getInstance().
getBufferPool().getNetDirectMemoryUsage();
if (useOffHeapForMerge == 1) {
ConcurrentMap<Long, Long> map = DbleServer.getInstance().
getServerMemory().
getResultMergeMemoryManager().getDirectMemoryUsage();
for (Map.Entry<Long, Long> entry : map.entrySet()) {
RowDataPacket row = new RowDataPacket(DETAIL_FIELD_COUNT);
long value = entry.getValue();
row.add(StringUtil.encode(String.valueOf(entry.getKey()), c.getCharset().getResults()));
/**
* DIRECTMEMORY used by result
*/
row.add(StringUtil.encode("MergeMemoryPool", c.getCharset().getResults()));
row.add(StringUtil.encode(value > 0 ? JavaUtils.bytesToString2(value) : "0", c.getCharset().getResults()));
row.setPacketId(++packetId);
buffer = row.write(buffer, c, true);
}
}
for (Map.Entry<Long, Long> entry : networkBufferPool.entrySet()) {
RowDataPacket row = new RowDataPacket(DETAIL_FIELD_COUNT);
long value = entry.getValue();
row.add(StringUtil.encode(String.valueOf(entry.getKey()), c.getCharset().getResults()));
/**
* DIRECTMEMORY belong to Buffer Pool
*/
/* DIRECT_MEMORY belong to Buffer Pool */
row.add(StringUtil.encode("NetWorkBufferPool", c.getCharset().getResults()));
row.add(StringUtil.encode(value > 0 ? JavaUtils.bytesToString2(value) : "0", c.getCharset().getResults()));
@@ -155,14 +123,13 @@ public final class ShowDirectMemory {
EOFPacket lastEof = new EOFPacket();
lastEof.setPacketId(++packetId);
buffer = lastEof.write(buffer, c, true);
// write buffer
c.write(buffer);
}
public static void showDirectMemoryTotal(ManagerConnection c) {
private static void showDirectMemoryTotal(ManagerConnection c) {
ByteBuffer buffer = c.allocate();
@@ -176,77 +143,22 @@ public final class ShowDirectMemory {
// write eof
buffer = TOTAL_EOF.write(buffer, c, true);
int useOffHeapForMerge = DbleServer.getInstance().getConfig().
getSystem().getUseOffHeapForMerge();
ConcurrentMap<Long, Long> networkBufferPool = DbleServer.getInstance().
getBufferPool().getNetDirectMemoryUsage();
RowDataPacket row = new RowDataPacket(TOTAL_FIELD_COUNT);
long usedForMerge = 0;
long usedForNetwork = 0;
/**
* the value of -XX:MaxDirectMemorySize
*/
row.add(StringUtil.encode(JavaUtils.bytesToString2(Platform.getMaxDirectMemory()), c.getCharset().getResults()));
if (useOffHeapForMerge == 1) {
/**
* used DirectMemory for merge
*/
ConcurrentMap<Long, Long> concurrentHashMap = DbleServer.getInstance().
getServerMemory().
getResultMergeMemoryManager().getDirectMemoryUsage();
for (Map.Entry<Long, Long> entry : concurrentHashMap.entrySet()) {
usedForMerge += entry.getValue();
}
}
/**
* IO packet used in DirectMemory in buffer pool
*/
/* the value of -XX:MaxDirectMemorySize */
long totalAvailable = Platform.getMaxDirectMemory();
row.add(StringUtil.encode(JavaUtils.bytesToString2(totalAvailable), c.getCharset().getResults()));
/* IO packet used in DirectMemory in buffer pool */
for (Map.Entry<Long, Long> entry : networkBufferPool.entrySet()) {
usedForNetwork += entry.getValue();
}
row.add(StringUtil.encode(JavaUtils.bytesToString2(usedForNetwork), c.getCharset().getResults()));
row.add(StringUtil.encode(JavaUtils.bytesToString2(totalAvailable - usedForNetwork), c.getCharset().getResults()));
row.add(StringUtil.encode(JavaUtils.bytesToString2(usedForMerge + usedForNetwork), c.getCharset().getResults()));
long totalAvailable = 0;
if (useOffHeapForMerge == 1) {
/**
* when use off-heap , avoid that MaxDirectMemorySize reached the limit of Physical memory.
* so the valid DirectMemory is MaxDirectMemorySize*DIRECT_SAFETY_FRACTION
*/
totalAvailable = (long) (Platform.getMaxDirectMemory() * SeverMemory.DIRECT_SAFETY_FRACTION);
} else {
totalAvailable = Platform.getMaxDirectMemory();
}
row.add(StringUtil.encode(JavaUtils.bytesToString2(totalAvailable - usedForMerge - usedForNetwork), c.getCharset().getResults()));
if (useOffHeapForMerge == 1) {
row.add(StringUtil.encode(("" + SeverMemory.DIRECT_SAFETY_FRACTION), c.getCharset().getResults()));
} else {
row.add(StringUtil.encode("1.0", c.getCharset().getResults()));
}
long reservedForOs = 0;
if (useOffHeapForMerge == 1) {
/**
* saved for OS
*/
reservedForOs = (long) ((1 - SeverMemory.DIRECT_SAFETY_FRACTION) *
(Platform.getMaxDirectMemory() -
2 * DbleServer.getInstance().getTotalNetWorkBufferSize()));
}
row.add(StringUtil.encode(reservedForOs > 0 ? JavaUtils.bytesToString2(reservedForOs) : "0", c.getCharset().getResults()));
// write rows
byte packetId = TOTAL_EOF.getPacketId();
row.setPacketId(++packetId);

View File

@@ -1,131 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory;
import com.actiontech.dble.config.model.SystemConfig;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.ResultMergeMemoryManager;
import com.actiontech.dble.memory.unsafe.storage.DataNodeDiskManager;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import com.google.common.annotations.VisibleForTesting;
import org.apache.log4j.Logger;
/**
* Created by zagnix on 2016/6/2. Memory manager used for:result,IO,system IO ia
* all IO Memory allocation use the Direct Memory
* result use Direct Memory and Heap
* System use the Heap Memory.
* JVM should set -XX:MaxDirectMemorySize and -Xmx -Xmn
* -Xss -XX:+UseParallelGC
*/
public class SeverMemory {
private static final Logger LOGGER = Logger.getLogger(SeverMemory.class);
public static final double DIRECT_SAFETY_FRACTION = 0.7;
private final long resultSetBufferSize;
private final int numCores;
private final ServerPropertyConf conf;
private final MemoryManager resultMergeMemoryManager;
private final DataNodeDiskManager blockManager;
private final SerializerManager serializerManager;
public SeverMemory(SystemConfig system, long totalNetWorkBufferSize)
throws NoSuchFieldException, IllegalAccessException {
LOGGER.info("useOffHeapForMerge = " + system.getUseOffHeapForMerge());
LOGGER.info("memoryPageSize = " + system.getMemoryPageSize());
LOGGER.info("spillsFileBufferSize = " + system.getSpillsFileBufferSize());
LOGGER.info("totalNetWorkBufferSize = " + JavaUtils.bytesToString2(totalNetWorkBufferSize));
LOGGER.info("dataNodeSortedTempDir = " + system.getDataNodeSortedTempDir());
this.conf = new ServerPropertyConf();
numCores = Runtime.getRuntime().availableProcessors();
resultSetBufferSize = (long) ((Platform.getMaxDirectMemory() - totalNetWorkBufferSize) * DIRECT_SAFETY_FRACTION);
assert resultSetBufferSize > 0;
if (system.getUseOffHeapForMerge() == 1) {
conf.set("server.memory.offHeap.enabled", "true");
} else {
conf.set("server.memory.offHeap.enabled", "false");
}
if (system.getMemoryPageSize() != null) {
conf.set("server.buffer.pageSize", system.getMemoryPageSize());
} else {
conf.set("server.buffer.pageSize", "1m");
}
if (system.getSpillsFileBufferSize() != null) {
conf.set("server.merge.file.buffer", system.getSpillsFileBufferSize());
} else {
conf.set("server.merge.file.buffer", "32k");
}
conf.set("server.local.dirs", system.getDataNodeSortedTempDir());
conf.set("server.pointer.array.len", "8k").set("server.memory.offHeap.size",
JavaUtils.bytesToString2(resultSetBufferSize));
LOGGER.info("resultSetBufferSize: " + JavaUtils.bytesToString2(resultSetBufferSize));
resultMergeMemoryManager = new ResultMergeMemoryManager(conf, numCores, 0);
serializerManager = new SerializerManager();
blockManager = new DataNodeDiskManager(conf, true);
}
@VisibleForTesting
public SeverMemory() throws NoSuchFieldException, IllegalAccessException {
conf = new ServerPropertyConf();
numCores = Runtime.getRuntime().availableProcessors();
long maxOnHeapMemory = (Platform.getMaxHeapMemory());
assert maxOnHeapMemory > 0;
resultSetBufferSize = (long) ((Platform.getMaxDirectMemory()) * DIRECT_SAFETY_FRACTION);
assert resultSetBufferSize > 0;
conf.set("server.memory.offHeap.enabled", "true").set("server.pointer.array.len", "8K").set("server.buffer.pageSize", "1m").set("server.memory.offHeap.size", JavaUtils.bytesToString2(resultSetBufferSize));
LOGGER.info("resultSetBufferSize: " + JavaUtils.bytesToString2(resultSetBufferSize));
resultMergeMemoryManager = new ResultMergeMemoryManager(conf, numCores, maxOnHeapMemory);
serializerManager = new SerializerManager();
blockManager = new DataNodeDiskManager(conf, true);
}
public ServerPropertyConf getConf() {
return conf;
}
public long getResultSetBufferSize() {
return resultSetBufferSize;
}
public MemoryManager getResultMergeMemoryManager() {
return resultMergeMemoryManager;
}
public SerializerManager getSerializerManager() {
return serializerManager;
}
public DataNodeDiskManager getBlockManager() {
return blockManager;
}
}

View File

@@ -1,129 +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 com.actiontech.dble.memory.environment;
import java.io.Serializable;
/**
* A hardware description describes the resources available to a task manager.
*/
public final class HardwareDescription implements Serializable {
private static final long serialVersionUID = 3380016608300325361L;
/**
* The number of CPU cores available to the JVM on the compute node.
*/
private int numberOfCPUCores;
/**
* The size of physical memory in bytes available on the compute node.
*/
private long sizeOfPhysicalMemory;
/**
* The size of the JVM heap memory
*/
private long sizeOfJvmHeap;
/**
* The size of the memory managed by the system for caching, hashing, sorting, ...
*/
private long sizeOfManagedMemory;
/**
* Public default constructor used for serialization process.
*/
public HardwareDescription() {
}
/**
* Constructs a new hardware description object.
*
* @param numberOfCPUCores The number of CPU cores available to the JVM on the compute node.
* @param sizeOfPhysicalMemory The size of physical memory in bytes available on the compute node.
* @param sizeOfJvmHeap The size of the JVM heap memory.
* @param sizeOfManagedMemory The size of the memory managed by the system for caching, hashing, sorting, ...
*/
public HardwareDescription(int numberOfCPUCores, long sizeOfPhysicalMemory, long sizeOfJvmHeap, long sizeOfManagedMemory) {
this.numberOfCPUCores = numberOfCPUCores;
this.sizeOfPhysicalMemory = sizeOfPhysicalMemory;
this.sizeOfJvmHeap = sizeOfJvmHeap;
this.sizeOfManagedMemory = sizeOfManagedMemory;
}
/**
* Returns the number of CPU cores available to the JVM on the compute node.
*
* @return the number of CPU cores available to the JVM on the compute node
*/
public int getNumberOfCPUCores() {
return this.numberOfCPUCores;
}
/**
* Returns the size of physical memory in bytes available on the compute node.
*
* @return the size of physical memory in bytes available on the compute node
*/
public long getSizeOfPhysicalMemory() {
return this.sizeOfPhysicalMemory;
}
/**
* Returns the size of the JVM heap memory
*
* @return The size of the JVM heap memory
*/
public long getSizeOfJvmHeap() {
return this.sizeOfJvmHeap;
}
/**
* Returns the size of the memory managed by the system for caching, hashing, sorting, ...
*
* @return The size of the memory managed by the system.
*/
public long getSizeOfManagedMemory() {
return this.sizeOfManagedMemory;
}
// --------------------------------------------------------------------------------------------
// Utils
// --------------------------------------------------------------------------------------------
@Override
public String toString() {
return String.format("cores=%d, physMem=%d, heap=%d, managed=%d",
numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, sizeOfManagedMemory);
}
// --------------------------------------------------------------------------------------------
// Factory
// --------------------------------------------------------------------------------------------
public static HardwareDescription extractFromSystem(long managedMemory) {
final int numberOfCPUCores = Hardware.getNumberCPUCores();
final long sizeOfJvmHeap = Runtime.getRuntime().maxMemory();
final long sizeOfPhysicalMemory = Hardware.getSizeOfPhysicalMemory();
return new HardwareDescription(numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, managedMemory);
}
}

View File

@@ -1,19 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe;
import java.io.IOException;
public abstract class KVIterator<K, V> {
public abstract boolean next() throws IOException;
public abstract K getKey();
public abstract V getValue();
public abstract void close();
}

View File

@@ -1,70 +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 com.actiontech.dble.memory.unsafe.array;
import com.actiontech.dble.memory.unsafe.Platform;
public final class ByteArrayMethods {
private ByteArrayMethods() {
// Private constructor, since this class only contains static methods.
}
/**
* Returns the next number greater or equal num that is power of 2.
*/
public static long nextPowerOf2(long num) {
final long highBit = Long.highestOneBit(num);
return (highBit == num) ? num : highBit << 1;
}
public static int roundNumberOfBytesToNearestWord(int numBytes) {
int remainder = numBytes & 0x07; // This is equivalent to `numBytes % 8`
if (remainder == 0) {
return numBytes;
} else {
return numBytes + (8 - remainder);
}
}
/**
* 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;
while (i <= length - 8) {
if (Platform.getLong(leftBase, leftOffset + i) !=
Platform.getLong(rightBase, rightOffset + i)) {
return false;
}
i += 8;
}
while (i < length) {
if (Platform.getByte(leftBase, leftOffset + i) !=
Platform.getByte(rightBase, rightOffset + i)) {
return false;
}
i += 1;
}
return true;
}
}

View File

@@ -1,89 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.array;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
/**
* @author Hash Zhang
* @version 1.0.0
* @date 2016/8/8
*/
public class CharArray {
private static final long WIDTH = 2;
private final MemoryBlock memory;
private final Object baseObj;
private final long baseOffset;
private final long length;
public CharArray(MemoryBlock memory) {
assert memory.size() < (long) Integer.MAX_VALUE * 2 : "Array size > 4 billion elements";
this.memory = memory;
this.baseObj = memory.getBaseObject();
this.baseOffset = memory.getBaseOffset();
this.length = memory.size() / WIDTH;
}
public MemoryBlock memoryBlock() {
return memory;
}
public Object getBaseObject() {
return baseObj;
}
public long getBaseOffset() {
return baseOffset;
}
/**
* Returns the number of elements this array can hold.
*/
public long size() {
return length;
}
/**
* Fill this all with 0L.
*/
public void zeroOut() {
for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) {
Platform.putLong(baseObj, off, 0);
}
}
/**
* Sets the value at position {@code index}.
*/
public void set(int index, char value) {
assert index >= 0 : "index (" + index + ") should >= 0";
assert index < length : "index (" + index + ") should < length (" + length + ")";
Platform.putChar(baseObj, baseOffset + index * WIDTH, value);
}
/**
* Returns the value at position {@code index}.
*/
public char get(int index) {
assert index >= 0 : "index (" + index + ") should >= 0";
assert index < length : "index (" + index + ") should < length (" + length + ")";
return Platform.getChar(baseObj, baseOffset + index * WIDTH);
}
public String toString() {
StringBuilder stringBuilder = new StringBuilder((int) this.length);
for (int i = 0; i < this.length; i++) {
stringBuilder.append(get(i));
}
return stringBuilder.toString();
}
//todo: from string
}

View File

@@ -1,96 +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 com.actiontech.dble.memory.unsafe.array;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
/**
* An array of long values. Compared with native JVM arrays, this:
* <ul>
* <li>supports using both in-heap and off-heap memory</li>
* <li>has no bound checking, and thus can crash the JVM process when assert is turned off</li>
* </ul>
*/
public final class LongArray {
// This is a long so that we perform long multiplications when computing offsets.
private static final long WIDTH = 8;
private final MemoryBlock memory;
private final Object baseObj;
private final long baseOffset;
private final long length;
public LongArray(MemoryBlock memory) {
assert memory.size() < (long) Integer.MAX_VALUE * 8 : "Array size > 4 billion elements";
this.memory = memory;
this.baseObj = memory.getBaseObject();
this.baseOffset = memory.getBaseOffset();
this.length = memory.size() / WIDTH;
}
public MemoryBlock memoryBlock() {
return memory;
}
public Object getBaseObject() {
return baseObj;
}
public long getBaseOffset() {
return baseOffset;
}
/**
* Returns the number of elements this array can hold.
*/
public long size() {
return length;
}
/**
* Fill this all with 0L.
*/
public void zeroOut() {
for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) {
Platform.putLong(baseObj, off, 0);
}
}
/**
* Sets the value at position {@code index}.
*/
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);
}
/**
* Returns the value at position {@code index}.
*/
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);
}
}

View File

@@ -1,131 +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 com.actiontech.dble.memory.unsafe.bitset;
import com.actiontech.dble.memory.unsafe.Platform;
/**
* Methods for working with fixed-size uncompressed bitsets.
* <p>
* We assume that the bitset data is word-aligned (that is, a multiple of 8 bytes in length).
* <p>
* Each bit occupies exactly one bit of storage.
*/
public final class BitSetMethods {
private static final long WORD_SIZE = 8;
private BitSetMethods() {
// Make the default constructor private, since this only holds static methods.
}
/**
* Sets the bit at the specified index to {@code true}.
*/
public static void set(Object baseObject, 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 = Platform.getLong(baseObject, wordOffset);
Platform.putLong(baseObject, wordOffset, word | mask);
}
/**
* Sets the bit at the specified index to {@code false}.
*/
public static void unset(Object baseObject, 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 = Platform.getLong(baseObject, wordOffset);
Platform.putLong(baseObject, wordOffset, word & ~mask);
}
/**
* Returns {@code true} if the bit is set at the specified index.
*/
public static boolean isSet(Object baseObject, 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 = Platform.getLong(baseObject, wordOffset);
return (word & mask) != 0;
}
/**
* Returns {@code true} if any bit is set.
*/
public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInWords) {
long address = baseOffset;
for (int i = 0; i < bitSetWidthInWords; i++, address += WORD_SIZE) {
if (Platform.getLong(baseObject, address) != 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.
* <p>
* To iterate over the true bits in a BitSet, use the following loop:
* <pre>
* <code>
* for (long i = bs.nextSetBit(0, sizeInWords); i &gt;= 0;
* i = bs.nextSetBit(i + 1, sizeInWords)) {
* // operate on index i here
* }
* </code>
* </pre>
*
* @param fromIndex the index to start checking from (inclusive)
* @param bitSetSizeInWords the size of the bitset, measured in 8-byte words
* @return the index of the next set bit, or -1 if there is no such bit
*/
public static int nextSetBit(
Object baseObject,
long baseOffset,
int fromIndex,
int bitSetSizeInWords) {
int wi = fromIndex >> 6;
if (wi >= bitSetSizeInWords) {
return -1;
}
// Try to find the next set bit in the current word
final int subIndex = fromIndex & 0x3f;
long word = Platform.getLong(baseObject, baseOffset + wi * WORD_SIZE) >> subIndex;
if (word != 0) {
return (wi << 6) + subIndex + Long.numberOfTrailingZeros(word);
}
// Find the next set bit in the rest of the words
wi += 1;
while (wi < bitSetSizeInWords) {
word = Platform.getLong(baseObject, baseOffset + wi * WORD_SIZE);
if (word != 0) {
return (wi << 6) + Long.numberOfTrailingZeros(word);
}
wi += 1;
}
return -1;
}
}

View File

@@ -1,127 +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 com.actiontech.dble.memory.unsafe.hash;
import com.actiontech.dble.memory.unsafe.Platform;
/**
* 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction.
*/
public final class Murmur3OfX86And32Bit {
private static final int C1 = 0xcc9e2d51;
private static final int C2 = 0x1b873593;
private final int seed;
public Murmur3OfX86And32Bit(int seed) {
this.seed = seed;
}
@Override
public String toString() {
return "Murmur3_32(seed=" + seed + ")";
}
public int hashInt(int input) {
return hashInt(input, seed);
}
public static int hashInt(int input, int seed) {
int k1 = mixK1(input);
int h1 = mixH1(seed, k1);
return fMix(h1, 4);
}
public int hashUnsafeWords(Object base, long offset, int lengthInBytes) {
return hashUnsafeWords(base, 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)";
int h1 = hashBytesByInt(base, offset, lengthInBytes, seed);
return fMix(h1, lengthInBytes);
}
public static int hashUnsafeBytes(Object 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);
}
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;
}
public int hashLong(long input) {
return hashLong(input, seed);
}
public static int hashLong(long input, int seed) {
int low = (int) input;
int high = (int) (input >>> 32);
int k1 = mixK1(low);
int h1 = mixH1(seed, k1);
k1 = mixK1(high);
h1 = mixH1(h1, k1);
return fMix(h1, 8);
}
private static int mixK1(int k1) {
k1 *= C1;
k1 = Integer.rotateLeft(k1, 15);
k1 *= C2;
return k1;
}
private static int mixH1(int h1, int k1) {
h1 ^= k1;
h1 = Integer.rotateLeft(h1, 13);
h1 = h1 * 5 + 0xe6546b64;
return h1;
}
// Finalization mix - force all bits of a hash block to avalanche
private static int fMix(int h1, int length) {
h1 ^= length;
h1 ^= h1 >>> 16;
h1 *= 0x85ebca6b;
h1 ^= h1 >>> 13;
h1 *= 0xc2b2ae35;
h1 ^= h1 >>> 16;
return h1;
}
}

View File

@@ -1,978 +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 com.actiontech.dble.memory.unsafe.map;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.ByteArrayMethods;
import com.actiontech.dble.memory.unsafe.array.LongArray;
import com.actiontech.dble.memory.unsafe.hash.Murmur3OfX86And32Bit;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryConsumer;
import com.actiontech.dble.memory.unsafe.storage.DataNodeDiskManager;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.actiontech.dble.memory.unsafe.utils.sort.UnsafeExternalSorter;
import com.actiontech.dble.memory.unsafe.utils.sort.UnsafeSorterSpillReader;
import com.actiontech.dble.memory.unsafe.utils.sort.UnsafeSorterSpillWriter;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.io.Closeables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Iterator;
import java.util.LinkedList;
/**
* An append-only hash map where keys and values are contiguous regions of bytes.
* <p>
* This is backed by a power-of-2-sized hash table, using quadratic probing with triangular numbers,
* which is guaranteed to exhaust the space.
* <p>
* The map can support up to 2^29 keys. If the key cardinality is higher than this, you should
* probably be using sorting instead of hashing for better cache locality.
* <p>
* The key and values under the hood are stored together, in the following format:
* Bytes 0 to 4: len(k) (key length in bytes) + len(v) (value length in bytes) + 4
* Bytes 4 to 8: len(k)
* Bytes 8 to 8 + len(k): key data
* Bytes 8 + len(k) to 8 + len(k) + len(v): value data
* Bytes 8 + len(k) + len(v) to 8 + len(k) + len(v) + 8: pointer to next pair
* <p>
* This means that the first four bytes store the entire record (key + value) length. This format
* is compatible with {@link UnsafeExternalSorter},
* so we can pass records from this map directly into the sorter to sort records in place.
*/
public final class BytesToBytesMap extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class);
private static final HashMapGrowthStrategy GROWTH_STRATEGY = HashMapGrowthStrategy.DOUBLING;
private final DataNodeMemoryManager dataNodeMemoryManager;
/**
* A linked list for tracking all allocated data pages so that we can free all of our memory.
*/
private final LinkedList<MemoryBlock> dataPages = new LinkedList<>();
/**
* The data page that will be used to store keys and values for new hashtable entries. When this
* page becomes full, a new page will be allocated and this pointer will change to point to that
* new page.
*/
private MemoryBlock currentPage = null;
/**
* Offset into `currentPage` that points to the location where new data can be inserted into
* the page. This does not incorporate the page's base offset.
*/
private long pageCursor = 0;
/**
* The maximum number of keys that BytesToBytesMap supports. The hash table has to be
* power-of-2-sized and its backing Java array can contain at most (1 &lt;&lt; 30) elements,
* since that's the largest power-of-2 that's less than Integer.MAX_VALUE. We need two long array
* entries per key, giving us a maximum capacity of (1 &lt;&lt; 29).
*/
@VisibleForTesting
public static final int MAX_CAPACITY = (1 << 29);
// This choice of page table size and page size means that we can address up to 500 gigabytes
// of memory.
/**
* A single array to store the key and value.
* <p>
* Position {@code 2 * i} in the array is used to track a pointer to the key at index {@code i},
* while position {@code 2 * i + 1} in the array holds key's full 32-bit hashcode.
*/
@Nullable
private LongArray longArray;
// TODO: we're wasting 32 bits of space here; we can probably store fewer bits of the hashcode
// and exploit word-alignment to use fewer bits to hold the address. This might let us store
// only one long per map entry, increasing the chance that this array will fit in cache at the
// expense of maybe performing more lookups if we have hash collisions. Say that we stored only
// 27 bits of the hashcode and 37 bits of the address. 37 bits is enough to address 1 terabyte
// of RAM given word-alignment. If we use 13 bits of this for our page table, that gives us a
// maximum page size of 2^24 * 8 = ~134 megabytes per page. This change will require us to store
// full base addresses in the page table for off-heap mode so that we can reconstruct the full
// absolute memory addresses.
/**
* Whether or not the longArray can grow. We will not insert more elements if it's false.
*/
private boolean canGrowArray = true;
private final double loadFactor;
/**
* The size of the data pages that hold key and value data. Map entries cannot span multiple
* pages, so this limits the maximum entry size.
*/
private final long pageSizeBytes;
/**
* Number of keys defined in the map.
*/
private int numKeys;
/**
* Number of values defined in the map. A key could have multiple values.
*/
private int numValues;
/**
* The map will be expanded once the number of keys exceeds this threshold.
*/
private int growthThreshold;
/**
* Mask for truncating hashcodes so that they do not exceed the long array's size.
* This is a strength reduction optimization; we're essentially performing a modulus operation,
* but doing so with a bitmask because this is a power-of-2-sized hash map.
*/
private int mask;
/**
* Return value of {@link BytesToBytesMap#lookup(Object, long, int)}.
*/
private final Location loc;
private final boolean enablePerfMetrics;
private long timeSpentResizingNs = 0;
private long numProbes = 0;
private long numKeyLookups = 0;
private long numHashCollisions = 0;
private long peakMemoryUsedBytes = 0L;
private final DataNodeDiskManager blockManager;
private final SerializerManager serializerManager;
private volatile MapIterator destructiveIterator = null;
private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
public BytesToBytesMap(
DataNodeMemoryManager dataNodeMemoryManager,
DataNodeDiskManager blockManager,
SerializerManager serializerManager,
int initialCapacity,
double loadFactor,
long pageSizeBytes,
boolean enablePerfMetrics) {
super(dataNodeMemoryManager, pageSizeBytes);
this.dataNodeMemoryManager = dataNodeMemoryManager;
this.blockManager = blockManager;
this.serializerManager = serializerManager;
this.loadFactor = loadFactor;
this.loc = new Location();
this.pageSizeBytes = pageSizeBytes;
this.enablePerfMetrics = enablePerfMetrics;
if (initialCapacity <= 0) {
throw new IllegalArgumentException("Initial capacity must be greater than 0");
}
if (initialCapacity > MAX_CAPACITY) {
throw new IllegalArgumentException(
"Initial capacity " + initialCapacity + " exceeds maximum capacity of " + MAX_CAPACITY);
}
if (pageSizeBytes > DataNodeMemoryManager.MAXIMUM_PAGE_SIZE_BYTES) {
throw new IllegalArgumentException("Page size " + pageSizeBytes + " cannot exceed " +
DataNodeMemoryManager.MAXIMUM_PAGE_SIZE_BYTES);
}
allocate(initialCapacity);
}
public BytesToBytesMap(
DataNodeMemoryManager dataNodeMemoryManager,
int initialCapacity,
long pageSizeBytes) {
this(dataNodeMemoryManager, initialCapacity, pageSizeBytes, false);
}
public BytesToBytesMap(
DataNodeMemoryManager dataNodeMemoryManager,
int initialCapacity,
long pageSizeBytes,
boolean enablePerfMetrics) {
this(
dataNodeMemoryManager,
null,
null,
initialCapacity,
0.70,
pageSizeBytes,
enablePerfMetrics);
}
/**
* Returns the number of keys defined in the map.
*/
public int numKeys() {
return numKeys;
}
/**
* Returns the number of values defined in the map. A key could have multiple values.
*/
public int numValues() {
return numValues;
}
/**
* Returns an iterator for iterating over the entries of this map.
* <p>
* For efficiency, all calls to `next()` will return the same {@link Location} object.
* <p>
* If any other lookups or operations are performed on this map while iterating over it, including
* `lookup()`, the behavior of the returned iterator is undefined.
*/
public MapIterator iterator() {
return new MapIterator(numValues, loc, false);
}
/**
* Returns a destructive iterator for iterating over the entries of this map. It frees each page
* as it moves onto next one. Notice: it is illegal to call any method on the map after
* `destructiveIterator()` has been called.
* <p>
* For efficiency, all calls to `next()` will return the same {@link Location} object.
* <p>
* If any other lookups or operations are performed on this map while iterating over it, including
* `lookup()`, the behavior of the returned iterator is undefined.
*/
public MapIterator destructiveIterator() {
return new MapIterator(numValues, loc, true);
}
/**
* Looks up a key, and return a {@link Location} handle that can be used to map existence
* and read/write values.
* <p>
* This function always return the same {@link Location} instance to avoid object allocation.
*/
public Location lookup(Object keyBase, long keyOffset, int keyLength) {
safeLookup(keyBase, keyOffset, keyLength, loc,
Murmur3OfX86And32Bit.hashUnsafeWords(keyBase, keyOffset, keyLength, 42));
return loc;
}
/**
* Looks up a key, and return a {@link Location} handle that can be used to map existence
* and read/write values.
* <p>
* This function always return the same {@link Location} instance to avoid object allocation.
*/
public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) {
safeLookup(keyBase, keyOffset, keyLength, loc, hash);
return loc;
}
/**
* Looks up a key, and saves the result in provided `loc`.
* <p>
* This is a thread-safe version of `lookup`, could be used by multiple threads.
*/
public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location location, int hash) {
assert (longArray != null);
if (enablePerfMetrics) {
numKeyLookups++;
}
int pos = hash & mask;
int step = 1;
while (true) {
if (enablePerfMetrics) {
numProbes++;
}
if (longArray.get(pos * 2) == 0) {
// This is a new key.
location.with(pos, hash, false);
return;
} else {
long stored = longArray.get(pos * 2 + 1);
/**
* hash is equal
*/
if ((int) (stored) == hash) {
// Full hash code matches.Let's compare the keys for equality.
location.with(pos, hash, true);
/**
* compare the key
*/
if (location.getKeyLength() == keyLength) {
final boolean areEqual = ByteArrayMethods.arrayEquals(
keyBase,
keyOffset,
location.getKeyBase(),
location.getKeyOffset(),
keyLength
);
if (areEqual) {
return;
} else {
if (enablePerfMetrics) {
numHashCollisions++;
}
}
}
}
}
pos = (pos + step) & mask;
step++;
}
}
/**
* Acquire a new page from the memory manager.
*
* @return whether there is enough space to allocate the new page.
*/
private boolean acquireNewPage(long required) {
try {
currentPage = allocatePage(required);
} catch (OutOfMemoryError e) {
return false;
}
dataPages.add(currentPage);
Platform.putInt(currentPage.getBaseObject(), currentPage.getBaseOffset(), 0);
pageCursor = 4;
return true;
}
@Override
public long spill(long size, MemoryConsumer trigger) throws IOException {
if (trigger != this && destructiveIterator != null) {
return destructiveIterator.spill(size);
}
return 0L;
}
/**
* Allocate new data structures for this map. When calling this outside of the constructor,
* make sure to keep references to the old data structures so that you can free them.
*
* @param capacity the new map capacity
*/
private void allocate(int capacity) {
assert (capacity >= 0);
capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64);
assert (capacity <= MAX_CAPACITY);
longArray = allocateLongArray(capacity * 2);
longArray.zeroOut();
this.growthThreshold = (int) (capacity * loadFactor);
this.mask = capacity - 1;
}
/**
* Free all allocated memory associated with this map, including the storage for keys and values
* as well as the hash map array itself.
* <p>
* This method is idempotent and can be called multiple times.
*/
public void free() {
updatePeakMemoryUsed();
if (longArray != null) {
freeLongArray(longArray);
longArray = null;
}
Iterator<MemoryBlock> dataPagesIterator = dataPages.iterator();
while (dataPagesIterator.hasNext()) {
MemoryBlock dataPage = dataPagesIterator.next();
dataPagesIterator.remove();
freePage(dataPage);
}
assert (dataPages.isEmpty());
while (!spillWriters.isEmpty()) {
File file = spillWriters.removeFirst().getFile();
if (file != null && file.exists()) {
if (!file.delete()) {
logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
}
}
}
}
public DataNodeMemoryManager getDataNodeMemoryManager() {
return dataNodeMemoryManager;
}
public long getPageSizeBytes() {
return pageSizeBytes;
}
/**
* Returns the total amount of memory, in bytes, consumed by this map's managed structures.
*/
public long getTotalMemoryConsumption() {
long totalDataPagesSize = 0L;
for (MemoryBlock dataPage : dataPages) {
totalDataPagesSize += dataPage.size();
}
return totalDataPagesSize + ((longArray != null) ? longArray.memoryBlock().size() : 0L);
}
private void updatePeakMemoryUsed() {
long mem = getTotalMemoryConsumption();
if (mem > peakMemoryUsedBytes) {
peakMemoryUsedBytes = mem;
}
}
/**
* Return the peak memory used so far, in bytes.
*/
public long getPeakMemoryUsedBytes() {
updatePeakMemoryUsed();
return peakMemoryUsedBytes;
}
/**
* Returns the total amount of time spent resizing this map (in nanoseconds).
*/
public long getTimeSpentResizingNs() {
if (!enablePerfMetrics) {
throw new IllegalStateException();
}
return timeSpentResizingNs;
}
/**
* Returns the average number of probes per key lookup.
*/
public double getAverageProbesPerLookup() {
if (!enablePerfMetrics) {
throw new IllegalStateException();
}
return (1.0 * numProbes) / numKeyLookups;
}
public long getNumHashCollisions() {
if (!enablePerfMetrics) {
throw new IllegalStateException();
}
return numHashCollisions;
}
@VisibleForTesting
public int getNumDataPages() {
return dataPages.size();
}
/**
* Returns the underline long[] of longArray.
*/
public LongArray getArray() {
assert (longArray != null);
return longArray;
}
/**
* Reset this map to initialized state.
*/
public void reset() {
numKeys = 0;
numValues = 0;
longArray.zeroOut();
while (dataPages.size() > 0) {
MemoryBlock dataPage = dataPages.removeLast();
freePage(dataPage);
}
currentPage = null;
pageCursor = 0;
}
/**
* Grows the size of the hash table and re-hash everything.
*/
@VisibleForTesting
void growAndRehash() {
assert (longArray != null);
long resizeStartTime = -1;
if (enablePerfMetrics) {
resizeStartTime = System.nanoTime();
}
// Store references to the old data structures to be used when we re-hash
final LongArray oldLongArray = longArray;
final int oldCapacity = (int) oldLongArray.size() / 2;
// Allocate the new data structures
allocate(Math.min(GROWTH_STRATEGY.nextCapacity(oldCapacity), MAX_CAPACITY));
// Re-mask (we don't recompute the hashcode because we stored all 32 bits of it)
for (int i = 0; i < oldLongArray.size(); i += 2) {
final long keyPointer = oldLongArray.get(i);
if (keyPointer == 0) {
continue;
}
final int hashcode = (int) oldLongArray.get(i + 1);
int newPos = hashcode & mask;
int step = 1;
while (longArray.get(newPos * 2) != 0) {
newPos = (newPos + step) & mask;
step++;
}
longArray.set(newPos * 2, keyPointer);
longArray.set(newPos * 2 + 1, hashcode);
}
freeLongArray(oldLongArray);
if (enablePerfMetrics) {
timeSpentResizingNs += System.nanoTime() - resizeStartTime;
}
}
/**
* Handle returned by {@link BytesToBytesMap#lookup(Object, long, int)} function.
*/
public final class Location {
/**
* An index into the hash map's Long array
*/
private int pos;
/**
* True if this location points to a position where a key is defined, false otherwise
*/
private boolean isDefined;
/**
* The hashcode of the most recent key passed to
* {@link BytesToBytesMap#lookup(Object, long, int, int)}. Caching this hashcode here allows us
* to avoid re-hashing the key when storing a value for that key.
*/
private int keyHashcode;
private Object baseObject; // the base object for key and value
private long keyOffset;
private int keyLength;
private long valueOffset;
private int valueLength;
/**
* Memory page containing the record. Only set if created by {@link BytesToBytesMap#iterator()}.
*/
@Nullable
private MemoryBlock memoryPage;
private void updateAddressesAndSizes(long fullKeyAddress) {
updateAddressesAndSizes(
dataNodeMemoryManager.getPage(fullKeyAddress),
dataNodeMemoryManager.getOffsetInPage(fullKeyAddress));
}
private void updateAddressesAndSizes(final Object base, long offset) {
baseObject = base;
final int totalLength = Platform.getInt(base, offset);
offset += 4;
keyLength = Platform.getInt(base, offset);
offset += 4;
keyOffset = offset;
valueOffset = offset + keyLength;
valueLength = totalLength - keyLength - 4;
}
private Location with(int position, int keyHash, boolean defined) {
assert (longArray != null);
this.pos = position;
this.isDefined = defined;
this.keyHashcode = keyHash;
if (defined) {
final long fullKeyAddress = longArray.get(position * 2);
updateAddressesAndSizes(fullKeyAddress);
}
return this;
}
private Location with(MemoryBlock page, long offsetInPage) {
this.isDefined = true;
this.memoryPage = page;
updateAddressesAndSizes(page.getBaseObject(), offsetInPage);
return this;
}
/**
* This is only used for spilling
*/
private Location with(Object base, long offset, int length) {
this.isDefined = true;
this.memoryPage = null;
baseObject = base;
keyOffset = offset + 4;
keyLength = Platform.getInt(base, offset);
valueOffset = offset + 4 + keyLength;
valueLength = length - 4 - keyLength;
return this;
}
/**
* Find the next pair that has the same key as current one.
*/
public boolean nextValue() {
assert isDefined;
long nextAddr = Platform.getLong(baseObject, valueOffset + valueLength);
if (nextAddr == 0) {
return false;
} else {
updateAddressesAndSizes(nextAddr);
return true;
}
}
/**
* Returns the memory page that contains the current record.
* This is only valid if this is returned by {@link BytesToBytesMap#iterator()}.
*/
public MemoryBlock getMemoryPage() {
return this.memoryPage;
}
/**
* Returns true if the key is defined at this position, and false otherwise.
*/
public boolean isDefined() {
return isDefined;
}
/**
* Returns the base object for key.
*/
public Object getKeyBase() {
assert (isDefined);
return baseObject;
}
/**
* Returns the offset for key.
*/
public long getKeyOffset() {
assert (isDefined);
return keyOffset;
}
/**
* Returns the base object for value.
*/
public Object getValueBase() {
assert (isDefined);
return baseObject;
}
/**
* Returns the offset for value.
*/
public long getValueOffset() {
assert (isDefined);
return valueOffset;
}
/**
* Returns the length of the key defined at this position.
* Unspecified behavior if the key is not defined.
*/
public int getKeyLength() {
assert (isDefined);
return keyLength;
}
/**
* Returns the length of the value defined at this position.
* Unspecified behavior if the key is not defined.
*/
public int getValueLength() {
assert (isDefined);
return valueLength;
}
/**
* Append a new value for the key. This method could be called multiple times for a given key.
* The return value indicates whether the put succeeded or whether it failed because additional
* memory could not be acquired.
* <p>
* It is only valid to call this method immediately after calling `lookup()` using the same key.
* </p>
* <p>
* The key and value must be word-aligned (that is, their sizes must multiples of 8).
* </p>
* <p>
* After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length`
* will return information on the data stored by this `append` call.
* </p>
* <p>
* As an example usage, here's the proper way to store a new key:
* </p>
* <pre>
* Location loc = map.lookup(keyBase, keyOffset, keyLength);
* if (!loc.isDefined()) {
* if (!loc.append(keyBase, keyOffset, keyLength, ...)) {
* // handle failure to grow map (by spilling, for example)
* }
* }
* </pre>
* <p>
* Unspecified behavior if the key is not defined.
* </p>
*
* @return true if the put() was successful and false if the put() failed because memory could
* not be acquired.
*/
public boolean append(Object kbase, long koff, int klen, Object vbase, long voff, int vlen) {
assert (klen % 8 == 0);
assert (vlen % 8 == 0);
assert (longArray != null);
if (numKeys == MAX_CAPACITY ||
// The map could be reused from last spill (because of no enough memory to grow),
// then we don't try to grow again if hit the `growthThreshold`.
!canGrowArray && numKeys > growthThreshold) {
return false;
}
// Here, we'll copy the data into our data pages. Because we only store a relative offset from
// the key address instead of storing the absolute address of the value, the key and value
// must be stored in the same memory page.
// (8 byte key length) (key) (value) (8 byte pointer to next value)
final long recordLength = 8 + klen + vlen + 8;
if (currentPage == null || currentPage.size() - pageCursor < recordLength) {
if (!acquireNewPage(recordLength + 4L)) {
return false;
}
}
// --- Append the key and value data to the current data page --------------------------------
final Object base = currentPage.getBaseObject();
long offset = currentPage.getBaseOffset() + pageCursor;
final long recordOffset = offset;
Platform.putInt(base, offset, klen + vlen + 4);
Platform.putInt(base, offset + 4, klen);
offset += 8;
Platform.copyMemory(kbase, koff, base, offset, klen);
offset += klen;
Platform.copyMemory(vbase, voff, base, offset, vlen);
offset += vlen;
// put this value at the beginning of the list
Platform.putLong(base, offset, isDefined ? longArray.get(pos * 2) : 0);
// --- Update bookkeeping data structures ----------------------------------------------------
offset = currentPage.getBaseOffset();
Platform.putInt(base, offset, Platform.getInt(base, offset) + 1);
pageCursor += recordLength;
final long storedKeyAddress = dataNodeMemoryManager.encodePageNumberAndOffset(
currentPage, recordOffset);
longArray.set(pos * 2, storedKeyAddress);
updateAddressesAndSizes(storedKeyAddress);
numValues++;
if (!isDefined) {
numKeys++;
longArray.set(pos * 2 + 1, keyHashcode);
isDefined = true;
if (numKeys > growthThreshold && longArray.size() < MAX_CAPACITY) {
try {
growAndRehash();
} catch (OutOfMemoryError oom) {
canGrowArray = false;
}
}
}
return true;
}
}
public final class MapIterator implements Iterator<Location> {
private int numRecords;
private final Location loc;
private MemoryBlock currentPage = null;
private int recordsInPage = 0;
private Object pageBaseObject;
private long offsetInPage;
// If this iterator destructive or not. When it is true, it frees each page as it moves onto
// next one.
private boolean destructive = false;
private UnsafeSorterSpillReader reader = null;
private MapIterator(int numRecords, Location loc, boolean destructive) {
this.numRecords = numRecords;
this.loc = loc;
this.destructive = destructive;
if (destructive) {
destructiveIterator = this;
}
}
private void advanceToNextPage() {
synchronized (this) {
int nextIdx = dataPages.indexOf(currentPage) + 1;
if (destructive && currentPage != null) {
dataPages.remove(currentPage);
freePage(currentPage);
nextIdx--;
}
if (dataPages.size() > nextIdx) {
currentPage = dataPages.get(nextIdx);
pageBaseObject = currentPage.getBaseObject();
offsetInPage = currentPage.getBaseOffset();
recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
offsetInPage += 4;
} else {
currentPage = null;
if (reader != null) {
// remove the spill file from disk
File file = spillWriters.removeFirst().getFile();
if (file != null && file.exists()) {
if (!file.delete()) {
logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
}
}
}
try {
Closeables.close(reader, /* swallowIOException = */ false);
if (spillWriters.size() > 0) {
reader = spillWriters.getFirst().getReader(serializerManager);
}
recordsInPage = -1;
} catch (IOException e) {
// Scala iterator does not handle exception
Platform.throwException(e);
}
}
}
}
@Override
public boolean hasNext() {
if (numRecords == 0) {
if (reader != null) {
// remove the spill file from disk
File file = spillWriters.removeFirst().getFile();
if (file != null && file.exists()) {
if (!file.delete()) {
logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
}
}
}
}
return numRecords > 0;
}
@Override
public Location next() {
if (recordsInPage == 0) {
advanceToNextPage();
}
numRecords--;
if (currentPage != null) {
int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
loc.with(currentPage, offsetInPage);
// [total size] [key size] [key] [value] [pointer to next]
offsetInPage += 4 + totalLength + 8;
recordsInPage--;
return loc;
} else {
assert (reader != null);
if (!reader.hasNext()) {
advanceToNextPage();
}
try {
reader.loadNext();
} catch (IOException e) {
try {
reader.close();
} catch (IOException e2) {
logger.error("Error while closing spill reader", e2);
}
// Scala iterator does not handle exception
Platform.throwException(e);
}
loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
return loc;
}
}
public long spill(long numBytes) throws IOException {
synchronized (this) {
if (!destructive || dataPages.size() == 1) {
return 0L;
}
long released = 0L;
while (dataPages.size() > 0) {
MemoryBlock block = dataPages.getLast();
// The currentPage is used, cannot be released
if (block == currentPage) {
break;
}
Object base = block.getBaseObject();
long offset = block.getBaseOffset();
int recordsNums = Platform.getInt(base, offset);
offset += 4;
final UnsafeSorterSpillWriter writer =
new UnsafeSorterSpillWriter(blockManager, 32 * 1024, recordsNums);
while (recordsNums > 0) {
int length = Platform.getInt(base, offset);
writer.write(base, offset + 4, length, 0);
offset += 4 + length + 8;
recordsNums--;
}
writer.close();
spillWriters.add(writer);
dataPages.removeLast();
released += block.size();
freePage(block);
if (released >= numBytes) {
break;
}
}
return released;
}
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
}

View File

@@ -1,41 +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 com.actiontech.dble.memory.unsafe.map;
/**
* Interface that defines how we can grow the size of a hash map when it is over a threshold.
*/
public interface HashMapGrowthStrategy {
int nextCapacity(int currentCapacity);
/**
* Double the size of the hash map every time.
*/
HashMapGrowthStrategy DOUBLING = new Doubling();
class Doubling implements HashMapGrowthStrategy {
@Override
public int nextCapacity(int currentCapacity) {
assert (currentCapacity > 0);
// Guard against overflow
return (currentCapacity * 2 > 0) ? (currentCapacity * 2) : Integer.MAX_VALUE;
}
}
}

View File

@@ -1,253 +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 com.actiontech.dble.memory.unsafe.map;
import com.actiontech.dble.memory.unsafe.KVIterator;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.hash.Murmur3OfX86And32Bit;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.row.StructType;
import com.actiontech.dble.memory.unsafe.row.UnsafeRow;
import org.apache.log4j.Logger;
/**
* Modify by zagnix ,add put find func
* Unsafe-based HashMap for performing aggregations where the aggregated values are fixed-width.
* This map supports a maximum of 2 billion keys.
*/
public final class UnsafeFixedWidthAggregationMap {
private static final Logger LOGGER = Logger.getLogger(UnsafeFixedWidthAggregationMap.class);
/**
* An empty aggregation buffer, encoded in UnsafeRow format. When inserting a new key into the
* map, we copy this buffer and use it as the value.
*/
private final byte[] emptyAggregationBuffer;
private final StructType aggregationBufferSchema;
private final StructType groupingKeySchema;
/**
* A hashmap which maps from opaque bytearray keys to bytearray values.
*/
private final BytesToBytesMap map;
/**
* Re-used pointer to the current aggregation buffer
*/
private final UnsafeRow currentAggregationBuffer;
private static final int SEED = 42;
/**
* @return true if UnsafeFixedWidthAggregationMap supports aggregation buffers with the given
* schema, false otherwise.
*/
public static boolean supportsAggregationBufferSchema(StructType schema) {
return true;
}
/**
* Create a new UnsafeFixedWidthAggregationMap.
*
* @param emptyAggregationBuffer the default value for new keys (a "zero" of the agg. function)
* @param aggregationBufferSchema the schema of the aggregation buffer, used for row conversion.
* @param groupingKeySchema the schema of the grouping key, used for row conversion.
* @param dataNodeMemoryManager the memory manager used to allocate our Unsafe memory structures.
* @param initialCapacity the initial capacity of the map (a sizing hint to avoid re-hashing).
* @param pageSizeBytes the data page size, in bytes; limits the maximum record size.
* @param enablePerfMetrics if true, performance metrics will be recorded (has minor perf impact)
*/
public UnsafeFixedWidthAggregationMap(
UnsafeRow emptyAggregationBuffer,
StructType aggregationBufferSchema,
StructType groupingKeySchema,
DataNodeMemoryManager dataNodeMemoryManager,
int initialCapacity,
long pageSizeBytes,
boolean enablePerfMetrics) {
this.aggregationBufferSchema = aggregationBufferSchema;
this.currentAggregationBuffer = new UnsafeRow(aggregationBufferSchema.length());
this.groupingKeySchema = groupingKeySchema;
this.map = new BytesToBytesMap(dataNodeMemoryManager, initialCapacity, pageSizeBytes, enablePerfMetrics);
boolean enablePerfMetrics1 = enablePerfMetrics;
this.emptyAggregationBuffer = emptyAggregationBuffer.getBytes();
}
/**
* Return the aggregation buffer for the current group. For efficiency, all calls to this method
* return the same object. If additional memory could not be allocated, then this method will
* signal an error by returning null.
*/
public UnsafeRow getAggregationBuffer(UnsafeRow groupingKey) {
return getAggregationBufferFromUnsafeRow(groupingKey);
}
public UnsafeRow getAggregationBufferFromUnsafeRow(UnsafeRow key) {
return getAggregationBufferFromUnsafeRow(key,
Murmur3OfX86And32Bit.hashUnsafeWords(key.getBaseObject(), key.getBaseOffset(),
key.getSizeInBytes(), SEED));
}
public UnsafeRow getAggregationBufferFromUnsafeRow(UnsafeRow key, int hash) {
// Probe our map using the serialized key
final BytesToBytesMap.Location loc = map.lookup(
key.getBaseObject(),
key.getBaseOffset(),
key.getSizeInBytes(),
hash);
if (!loc.isDefined()) {
// This is the first time that we've seen this grouping key, so we'll insert a copy of the
// empty aggregation buffer into the map:
boolean putSucceeded = loc.append(
key.getBaseObject(),
key.getBaseOffset(),
key.getSizeInBytes(),
emptyAggregationBuffer,
Platform.BYTE_ARRAY_OFFSET,
emptyAggregationBuffer.length
);
if (!putSucceeded) {
return null;
}
}
// Reset the pointer to point to the value that we just stored or looked up:
currentAggregationBuffer.pointTo(
loc.getValueBase(),
loc.getValueOffset(),
loc.getValueLength()
);
return currentAggregationBuffer;
}
public boolean put(UnsafeRow key, UnsafeRow value) {
int hash = Murmur3OfX86And32Bit.hashUnsafeWords(key.getBaseObject(),
key.getBaseOffset(), key.getSizeInBytes(), SEED);
// Probe our map using the serialized key
final BytesToBytesMap.Location loc = map.lookup(
key.getBaseObject(),
key.getBaseOffset(),
key.getSizeInBytes(),
hash);
if (!loc.isDefined()) {
// This is the first time that we've seen this grouping key, so we'll insert a copy of the
// empty aggregation buffer into the map:
boolean putSucceeded = loc.append(
key.getBaseObject(),
key.getBaseOffset(),
key.getSizeInBytes(),
value.getBaseObject(),
value.getBaseOffset(),
value.getSizeInBytes());
if (!putSucceeded) {
return false;
}
}
return true;
}
public boolean find(UnsafeRow key) {
int hash = Murmur3OfX86And32Bit.hashUnsafeWords(key.getBaseObject(), key.getBaseOffset(), key.getSizeInBytes(), 42);
// Probe our map using the serialized key
final BytesToBytesMap.Location loc = map.lookup(key.getBaseObject(),
key.getBaseOffset(), key.getSizeInBytes(), hash);
return loc.isDefined();
}
/**
* Returns an iterator over the keys and values in this map. This uses destructive iterator of
* BytesToBytesMap. So it is illegal to call any other method on this map after `iterator()` has
* been called.
* <p>
* For efficiency, each call returns the same object.
*/
public KVIterator<UnsafeRow, UnsafeRow> iterator() {
return new KVIterator<UnsafeRow, UnsafeRow>() {
private final BytesToBytesMap.MapIterator mapLocationIterator = map.iterator();
private final UnsafeRow key = new UnsafeRow(groupingKeySchema.length());
private final UnsafeRow value = new UnsafeRow(aggregationBufferSchema.length());
@Override
public boolean next() {
if (mapLocationIterator.hasNext()) {
final BytesToBytesMap.Location loc = mapLocationIterator.next();
if (loc == null)
return false;
key.pointTo(
loc.getKeyBase(),
loc.getKeyOffset(),
loc.getKeyLength()
);
value.pointTo(
loc.getValueBase(),
loc.getValueOffset(),
loc.getValueLength()
);
return true;
} else {
return false;
}
}
@Override
public UnsafeRow getKey() {
return key;
}
@Override
public UnsafeRow getValue() {
return value;
}
@Override
public void close() {
}
};
}
/**
* Return the peak memory used so far, in bytes.
*/
public long getPeakMemoryUsedBytes() {
return map.getPeakMemoryUsedBytes();
}
/**
* Free the memory associated with this map. This is idempotent and can be called multiple times.
*/
public void free() {
map.free();
}
}

View File

@@ -1,88 +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 com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.Platform;
import javax.annotation.concurrent.GuardedBy;
import java.lang.ref.WeakReference;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.Map;
/**
* A simple {@link MemoryAllocator} that can allocate up to 16GB using a JVM long primitive array.
*/
public class HeapMemoryAllocator implements MemoryAllocator {
@GuardedBy("this")
private final Map<Long, LinkedList<WeakReference<MemoryBlock>>> bufferPoolsBySize =
new HashMap<>();
private static final int POOLING_THRESHOLD_BYTES = 1024 * 1024;
/**
* Returns true if allocations of the given size should go through the pooling mechanism and
* false otherwise.
*/
private boolean shouldPool(long size) {
// Very small allocations are less likely to benefit from pooling.
return size >= POOLING_THRESHOLD_BYTES;
}
@Override
public MemoryBlock allocate(long size) throws OutOfMemoryError {
if (shouldPool(size)) {
synchronized (this) {
final LinkedList<WeakReference<MemoryBlock>> pool = bufferPoolsBySize.get(size);
if (pool != null) {
while (!pool.isEmpty()) {
final WeakReference<MemoryBlock> blockReference = pool.pop();
final MemoryBlock memory = blockReference.get();
if (memory != null) {
assert (memory.size() == size);
return memory;
}
}
bufferPoolsBySize.remove(size);
}
}
}
long[] array = new long[(int) ((size + 7) / 8)];
return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size);
}
@Override
public void free(MemoryBlock memory) {
final long size = memory.size();
if (shouldPool(size)) {
synchronized (this) {
LinkedList<WeakReference<MemoryBlock>> pool = bufferPoolsBySize.get(size);
if (pool == null) {
pool = new LinkedList<>();
bufferPoolsBySize.put(size, pool);
}
pool.add(new WeakReference<>(memory));
}
} else {
// Do nothing
}
}
}

View File

@@ -1,33 +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 com.actiontech.dble.memory.unsafe.memory;
public interface MemoryAllocator {
/**
* Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed
* to be zeroed out (call `zero()` on the result if this is necessary).
*/
MemoryBlock allocate(long size) throws OutOfMemoryError;
void free(MemoryBlock memory);
MemoryAllocator UNSAFE = new UnsafeMemoryAllocator();
MemoryAllocator HEAP = new HeapMemoryAllocator();
}

View File

@@ -1,66 +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 com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.Platform;
import javax.annotation.Nullable;
/**
* A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size.
*/
public class MemoryBlock extends MemoryLocation {
private final long length;
private int pageNumber = -1;
public MemoryBlock(@Nullable Object obj, long offset, long length) {
super(obj, offset);
this.length = length;
}
/**
* Returns the size of the memory block.
*/
public long size() {
return length;
}
/**
* Creates a memory block pointing to the memory used by the long array.
*/
public static MemoryBlock fromLongArray(final long[] array) {
return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8);
}
/**
* Optional page number; used when this MemoryBlock represents a page allocated by a
* DataNodeMemoryManager. This field is public so that it can be modified by the DataNodeMemoryManager,
* which lives in a different package.
*/
public int getPageNumber() {
return pageNumber;
}
public void setPageNumber(int pageNumber) {
this.pageNumber = pageNumber;
}
}

View File

@@ -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 com.actiontech.dble.memory.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;
}
}

View File

@@ -1,40 +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 com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.Platform;
/**
* A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory.
*/
public class UnsafeMemoryAllocator implements MemoryAllocator {
@Override
public MemoryBlock allocate(long size) throws OutOfMemoryError {
long address = Platform.allocateMemory(size);
return new MemoryBlock(null, address, size);
}
@Override
public void free(MemoryBlock memory) {
assert (memory.obj == null) :
"baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?";
Platform.freeMemory(memory.offset);
}
}

View File

@@ -1,431 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory.mm;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.concurrent.GuardedBy;
import java.io.IOException;
import java.util.Arrays;
import java.util.BitSet;
import java.util.HashSet;
/**
* Modify by zagnix
* Manages the memory allocated by an individual thread.
* <p>
* Most of the complexity in this class deals with encoding of off-heap addresses into 64-bit longs.
* In off-heap mode, memory can be directly addressed with 64-bit longs. In on-heap mode, memory is
* addressed by the combination of a base Object reference and a 64-bit offset within that object.
* This is a problem when we want to store pointers to data structures inside of other structures,
* such as record pointers inside hashmaps or sorting buffers. Even if we decided to use 128 bits
* to address memory, we can't just store the address of the base object since it's not guaranteed
* to remain stable as the heap gets reorganized due to GC.
* <p>
* Instead, we use the following approach to encode record pointers in 64-bit longs: for off-heap
* mode, just store the raw address, and for on-heap mode use the upper 13 bits of the address to
* store a "page number" and the lower 51 bits to store an offset within this page. These page
* numbers are used to index into a "page table" array inside of the MemoryManager in order to
* retrieve the base object.
* <p>
* This allows us to address 8192 pages. In on-heap mode, the maximum page size is limited by the
* maximum size of a long[] array, allowing us to address 8192 * 2^32 * 8 bytes, which is
* approximately 35 terabytes of memory.
*/
public class DataNodeMemoryManager {
private final Logger logger = LoggerFactory.getLogger(DataNodeMemoryManager.class);
/**
* The number of bits used to address the page table.
*/
private static final int PAGE_NUMBER_BITS = 13;
/**
* The number of bits used to encode offsets in data pages.
*/
public static final int OFFSET_BITS = 64 - PAGE_NUMBER_BITS; // 51
/**
* The number of entries in the page table.
*/
private static final int PAGE_TABLE_SIZE = 1 << PAGE_NUMBER_BITS;
/**
* Maximum supported data page size (in bytes). In principle, the maximum addressable page size is
* (1L &lt;&lt; OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's
* maximum page size is limited by the maximum amount of data that can be stored in a long[]
* array, which is (2^32 - 1) * 8 bytes (or 16 gigabytes). Therefore, we cap this at 16 gigabytes.
*/
public static final long MAXIMUM_PAGE_SIZE_BYTES = ((1L << 31) - 1) * 8L;
/**
* Bit mask for the lower 51 bits of a long.
*/
private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL;
/**
* Bit mask for the upper 13 bits of a long
*/
private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS;
/**
* Similar to an operating system's page table, this array maps page numbers into base object
* pointers, allowing us to translate between the hashtable's internal 64-bit address
* representation and the baseObject+offset representation which we use to support both in- and
* off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`.
* When using an in-heap allocator, the entries in this map will point to pages' base objects.
* Entries are added to this map as new data pages are allocated.
*/
private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE];
/**
* Bitmap for tracking free pages.
*/
private final BitSet allocatedPages = new BitSet(PAGE_TABLE_SIZE);
private final MemoryManager memoryManager;
private final long connectionAttemptId;
/**
* Tracks whether we're in-heap or off-heap. For off-heap, we short-circuit most of these methods
* without doing any masking or lookups. Since this branching should be well-predicted by the JIT,
* this extra layer of indirection / abstraction hopefully shouldn't be too expensive.
*/
public final MemoryMode tungstenMemoryMode;
/**
* Tracks spillable memory consumers.
*/
@GuardedBy("this")
private final HashSet<MemoryConsumer> consumers;
/**
* The amount of memory that is acquired but not used.
*/
private volatile long acquiredButNotUsed = 0L;
/**
* Construct a new DataNodeMemoryManager.
*/
public DataNodeMemoryManager(MemoryManager memoryManager, long connectionAttemptId) {
this.tungstenMemoryMode = memoryManager.tungstenMemoryMode();
this.memoryManager = memoryManager;
this.connectionAttemptId = connectionAttemptId;
this.consumers = new HashSet<>();
}
/**
* Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
* spill() of consumers to release more memory.
*
* @return number of bytes successfully granted (<= N).
*/
public long acquireExecutionMemory(long required, MemoryMode mode, MemoryConsumer consumer) throws InterruptedException {
assert (required >= 0);
// If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap
// memory here, then it may not make sense to spill since that would only end up freeing
// off-heap memory. This is subject to change, though, so it may be risky to make this
// optimization now in case we forget to undo it late when making changes.
synchronized (this) {
long got = memoryManager.acquireExecutionMemory(required, connectionAttemptId, mode);
// Try to release memory from other consumers first, then we can reduce the frequency of
// spilling, avoid to have too many spilled files.
if (got < required) {
// Call spill() on other consumers to release memory
for (MemoryConsumer c : consumers) {
if (c != consumer && c.getUsed() > 0) {
try {
/**
* write data to disk
*/
long released = c.spill(required - got, consumer);
if (released > 0 && mode == tungstenMemoryMode) {
logger.info("Thread " + connectionAttemptId + " released " + JavaUtils.bytesToString(released) +
" from " + c + " for" + consumer);
got += memoryManager.acquireExecutionMemory(required - got, connectionAttemptId, mode);
if (got >= required) {
break;
}
}
} catch (IOException e) {
logger.error("error while calling spill() on " + c, e);
throw new OutOfMemoryError("error while calling spill() on " + c + " : " + e.getMessage());
}
}
}
}
// call spill() on itself
if (got < required && consumer != null) {
try {
long released = consumer.spill(required - got, consumer);
if (released > 0 && mode == tungstenMemoryMode) {
logger.info("Thread " + connectionAttemptId +
" released " + JavaUtils.bytesToString(released) + "from itself (" + consumer + ")");
got += memoryManager.acquireExecutionMemory(required - got, connectionAttemptId, mode);
}
} catch (IOException e) {
logger.error("error while calling spill() on " + consumer, e);
throw new OutOfMemoryError("error while calling spill() on " + consumer + " : " + e.getMessage());
}
}
if (consumer != null) {
consumers.add(consumer);
}
// logger.info("Thread" + connectionAttemptId + " acquire "+ JavaUtils.bytesToString(got) +" for "+ consumer+"");
return got;
}
}
/**
* Release N bytes of execution memory for a MemoryConsumer.
*/
public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer consumer) {
logger.debug("Thread" + connectionAttemptId + " release " + JavaUtils.bytesToString(size) + " from " + consumer + "");
memoryManager.releaseExecutionMemory(size, connectionAttemptId, mode);
}
/**
* Dump the memory usage of all consumers.
*/
public void showMemoryUsage() {
logger.info("Memory used in Thread " + connectionAttemptId);
synchronized (this) {
long memoryAccountedForByConsumers = 0;
for (MemoryConsumer c : consumers) {
long totalMemUsage = c.getUsed();
memoryAccountedForByConsumers += totalMemUsage;
if (totalMemUsage > 0) {
logger.info("Acquired by " + c + ": " + JavaUtils.bytesToString(totalMemUsage));
}
}
long memoryNotAccountedFor =
memoryManager.getExecutionMemoryUsageForConnection(connectionAttemptId) - memoryAccountedForByConsumers;
logger.info(
"{} bytes of memory were used by task {} but are not associated with specific consumers",
memoryNotAccountedFor, connectionAttemptId);
logger.info(
"{} bytes of memory are used for execution and {} bytes of memory are used for storage",
memoryManager.executionMemoryUsed());
}
}
/**
* Return the page size in bytes.
*/
public long pageSizeBytes() {
return memoryManager.pageSizeBytes();
}
/**
* Allocate a block of memory that will be tracked in the MemoryManager's page table; this is
* intended for allocating large blocks of Tungsten memory that will be shared between operators.
* <p>
* Returns `null` if there was not enough memory to allocate the page. May return a page that
* contains fewer bytes than requested, so callers should verify the size of returned pages.
*/
public MemoryBlock allocatePage(long size, MemoryConsumer consumer) {
if (size > MAXIMUM_PAGE_SIZE_BYTES) {
throw new IllegalArgumentException(
"Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes");
}
/**
* spill to disk ,release the memory
*/
long acquired = 0;
try {
acquired = acquireExecutionMemory(size, tungstenMemoryMode, consumer);
} catch (InterruptedException e) {
logger.error(e.getMessage());
}
if (acquired <= 0) {
return null;
}
final int pageNumber;
synchronized (this) {
pageNumber = allocatedPages.nextClearBit(0);
if (pageNumber >= PAGE_TABLE_SIZE) {
releaseExecutionMemory(acquired, tungstenMemoryMode, consumer);
throw new IllegalStateException(
"Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages");
}
allocatedPages.set(pageNumber);
}
MemoryBlock page = null;
try {
page = memoryManager.tungstenMemoryAllocator().allocate(acquired);
} catch (OutOfMemoryError e) {
logger.warn("Failed to allocate a page ({} bytes), try again.", acquired);
// there is no enough memory actually, it means the actual free memory is smaller than
// MemoryManager thought, we should keep the acquired memory.
synchronized (this) {
acquiredButNotUsed += acquired;
allocatedPages.clear(pageNumber);
}
// this could trigger spilling to free some pages.
return allocatePage(size, consumer);
}
page.setPageNumber(pageNumber);
pageTable[pageNumber] = page;
return page;
}
/**
* Free a block of memory allocated via {@link DataNodeMemoryManager#allocatePage}.
*/
public void freePage(MemoryBlock page, MemoryConsumer consumer) {
assert (page.getPageNumber() != -1) :
"Called freePage() on memory that wasn't allocated with allocatePage()";
assert (allocatedPages.get(page.getPageNumber()));
pageTable[page.getPageNumber()] = null;
synchronized (this) {
allocatedPages.clear(page.getPageNumber());
}
logger.trace("Freed page number " + page.getPageNumber() + " (" + page.size() + " bytes)");
long pageSize = page.size();
memoryManager.tungstenMemoryAllocator().free(page);
releaseExecutionMemory(pageSize, tungstenMemoryMode, consumer);
}
/**
* Given a memory page and offset within that page, encode this address into a 64-bit long.
* This address will remain valid as long as the corresponding page has not been freed.
*
* @param page a data page allocated by {@link DataNodeMemoryManager#allocatePage}/
* @param offsetInPage an offset in this page which incorporates the base offset. In other words,
* this should be the value that you would pass as the base offset into an
* UNSAFE call (e.g. page.baseOffset() + something).
* @return an encoded page address.
*/
public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) {
if (tungstenMemoryMode == MemoryMode.OFF_HEAP) {
// In off-heap mode, an offset is an absolute address that may require a full 64 bits to
// encode. Due to our page size limitation, though, we can convert this into an offset that's
// relative to the page's base offset; this relative offset will fit in 51 bits.
offsetInPage -= page.getBaseOffset();
}
return encodePageNumberAndOffset(page.getPageNumber(), offsetInPage);
}
@VisibleForTesting
public static long encodePageNumberAndOffset(int pageNumber, long offsetInPage) {
assert (pageNumber != -1) : "encodePageNumberAndOffset called with invalid page";
return (((long) pageNumber) << OFFSET_BITS) | (offsetInPage & MASK_LONG_LOWER_51_BITS);
}
@VisibleForTesting
public static int decodePageNumber(long pagePlusOffsetAddress) {
return (int) (pagePlusOffsetAddress >>> OFFSET_BITS);
}
private static long decodeOffset(long pagePlusOffsetAddress) {
return (pagePlusOffsetAddress & MASK_LONG_LOWER_51_BITS);
}
/**
* Get the page associated with an address encoded by
* {@link DataNodeMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)}
*/
public Object getPage(long pagePlusOffsetAddress) {
if (tungstenMemoryMode == MemoryMode.ON_HEAP) {
final int pageNumber = decodePageNumber(pagePlusOffsetAddress);
assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE);
final MemoryBlock page = pageTable[pageNumber];
assert (page != null);
assert (page.getBaseObject() != null);
return page.getBaseObject();
} else {
return null;
}
}
/**
* Get the offset associated with an address encoded by
* {@link DataNodeMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)}
*/
public long getOffsetInPage(long pagePlusOffsetAddress) {
final long offsetInPage = decodeOffset(pagePlusOffsetAddress);
if (tungstenMemoryMode == MemoryMode.ON_HEAP) {
return offsetInPage;
} else {
// In off-heap mode, an offset is an absolute address. In encodePageNumberAndOffset, we
// converted the absolute address into a relative address. Here, we invert that operation:
final int pageNumber = decodePageNumber(pagePlusOffsetAddress);
assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE);
final MemoryBlock page = pageTable[pageNumber];
assert (page != null);
return page.getBaseOffset() + offsetInPage;
}
}
/**
* Clean up all allocated memory and pages. Returns the number of bytes freed. A non-zero return
* value can be used to detect memory leaks.
*/
public long cleanUpAllAllocatedMemory() {
synchronized (this) {
for (MemoryConsumer c : consumers) {
if (c != null && c.getUsed() > 0) {
// In case of failed task, it's normal to see leaked memory
logger.warn("leak " + JavaUtils.bytesToString(c.getUsed()) + " memory from " + c);
}
}
consumers.clear();
for (MemoryBlock page : pageTable) {
if (page != null) {
logger.warn("leak a page: " + page + " in task " + connectionAttemptId);
memoryManager.tungstenMemoryAllocator().free(page);
}
}
Arrays.fill(pageTable, null);
}
// release the memory that is not used by any consumer.
memoryManager.releaseExecutionMemory(acquiredButNotUsed, connectionAttemptId, tungstenMemoryMode);
return memoryManager.releaseAllExecutionMemoryForConnection(connectionAttemptId);
}
/**
* Returns the memory consumption, in bytes, for the current task.
*/
public long getMemoryConsumptionForThisConnection() {
return memoryManager.getExecutionMemoryUsageForConnection(connectionAttemptId);
}
/**
* Returns Tungsten memory mode
*/
public MemoryMode getTungstenMemoryMode() {
return tungstenMemoryMode;
}
}

View File

@@ -1,180 +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 com.actiontech.dble.memory.unsafe.memory.mm;
import com.actiontech.dble.memory.unsafe.array.CharArray;
import com.actiontech.dble.memory.unsafe.array.LongArray;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
/**
* An memory consumer of DataNodeMemoryManager, which support spilling.
* Note: this only supports allocation / spilling of Tungsten memory.
*/
public abstract class MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(MemoryConsumer.class);
protected final DataNodeMemoryManager dataNodeMemoryManager;
private final long pageSize;
protected long used;
protected MemoryConsumer(DataNodeMemoryManager dataNodeMemoryManager, long pageSize) {
this.dataNodeMemoryManager = dataNodeMemoryManager;
this.pageSize = pageSize;
}
protected MemoryConsumer(DataNodeMemoryManager dataNodeMemoryManager) {
this(dataNodeMemoryManager, dataNodeMemoryManager.pageSizeBytes());
}
/**
* Returns the size of used memory in bytes.
*/
public long getUsed() {
return used;
}
/**
* Force spill during building.
* <p>
* For testing.
*/
public void spill() throws IOException {
spill(Long.MAX_VALUE, this);
}
/**
* Spill some data to disk to release memory, which will be called by DataNodeMemoryManager
* when there is not enough memory for the task.
* <p>
* This should be implemented by subclass.
* <p>
* Note: In order to avoid possible deadlock, should not call acquireMemory() from spill().
* <p>
* Note: today, this only frees Tungsten-managed pages.
*
* @param size the amount of memory should be released
* @param trigger the MemoryConsumer that trigger this spilling
* @return the amount of released memory in bytes
* @throws IOException
*/
public abstract long spill(long size, MemoryConsumer trigger) throws IOException;
/**
* Allocates a LongArray of `size`.
*/
public LongArray allocateLongArray(long size) {
long required = size * 8L;
MemoryBlock page = dataNodeMemoryManager.allocatePage(required, this);
if (page == null || page.size() < required) {
long got = 0;
if (page != null) {
got = page.size();
dataNodeMemoryManager.freePage(page, this);
}
dataNodeMemoryManager.showMemoryUsage();
throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
}
used += required;
return new LongArray(page);
}
/**
* Frees a LongArray.
*/
public void freeLongArray(LongArray array) {
freePage(array.memoryBlock());
}
public CharArray allocateCharArray(long size) {
long required = size * 2L;
MemoryBlock page = dataNodeMemoryManager.allocatePage(required, this);
if (page == null || page.size() < required) {
long got = 0;
if (page != null) {
got = page.size();
dataNodeMemoryManager.freePage(page, this);
}
dataNodeMemoryManager.showMemoryUsage();
throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
}
used += required;
return new CharArray(page);
}
/**
* Frees a CharArray.
*/
public void freeCharArray(CharArray array) {
freePage(array.memoryBlock());
}
/**
* Allocate a memory block with at least `required` bytes.
* <p>
* Throws IOException if there is not enough memory.
*
* @throws OutOfMemoryError
*/
protected MemoryBlock allocatePage(long required) {
MemoryBlock page = dataNodeMemoryManager.allocatePage(Math.max(pageSize, required), this);
if (page == null || page.size() < required) {
long got = 0;
if (page != null) {
got = page.size();
dataNodeMemoryManager.freePage(page, this);
}
dataNodeMemoryManager.showMemoryUsage();
throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
}
used += page.size();
return page;
}
/**
* Free a memory block.
*/
protected void freePage(MemoryBlock page) {
used -= page.size();
dataNodeMemoryManager.freePage(page, this);
}
/**
* Allocates a heap memory of `size`.
*/
public long acquireOnHeapMemory(long size) {
long granted = 0;
try {
granted = dataNodeMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this);
} catch (InterruptedException e) {
logger.error(e.getMessage());
}
used += granted;
return granted;
}
/**
* Release N bytes of heap memory.
*/
public void freeOnHeapMemory(long size) {
dataNodeMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this);
used -= size;
}
}

View File

@@ -1,172 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory.mm;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.ByteArrayMethods;
import com.actiontech.dble.memory.unsafe.memory.MemoryAllocator;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import javax.annotation.concurrent.GuardedBy;
import java.util.concurrent.ConcurrentMap;
public abstract class MemoryManager {
private ServerPropertyConf conf;
@GuardedBy("this")
protected ResultSetMemoryPool onHeapExecutionMemoryPool =
new ResultSetMemoryPool(this, MemoryMode.ON_HEAP);
@GuardedBy("this")
protected ResultSetMemoryPool offHeapExecutionMemoryPool =
new ResultSetMemoryPool(this, MemoryMode.OFF_HEAP);
protected long maxOffHeapMemory = 0L;
protected long offHeapExecutionMemory = 0L;
private int numCores = 0;
public MemoryManager(ServerPropertyConf conf, int numCores, long onHeapExecutionMemory) {
this.conf = conf;
this.numCores = numCores;
maxOffHeapMemory = conf.getSizeAsBytes("server.memory.offHeap.size", "128m");
offHeapExecutionMemory = maxOffHeapMemory;
onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory);
offHeapExecutionMemoryPool.incrementPoolSize(offHeapExecutionMemory);
}
protected abstract long acquireExecutionMemory(long numBytes, long taskAttemptId, MemoryMode memoryMode) throws InterruptedException;
/**
* Release numBytes of execution memory belonging to the given task.
*/
public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode memoryMode) {
synchronized (this) {
if (memoryMode == MemoryMode.ON_HEAP) {
onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
} else if (memoryMode == MemoryMode.OFF_HEAP) {
offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
}
}
}
/**
* Release all memory for the given task and mark it as inactive (e.g. when a task ends).
*
* @return the number of bytes freed.
*/
public long releaseAllExecutionMemoryForConnection(long connAttemptId) {
synchronized (this) {
return (onHeapExecutionMemoryPool.releaseAllMemoryForeConnection(connAttemptId) +
offHeapExecutionMemoryPool.releaseAllMemoryForeConnection(connAttemptId));
}
}
/**
* Execution memory currently in use, in bytes.
*/
public final long executionMemoryUsed() {
synchronized (this) {
return (onHeapExecutionMemoryPool.memoryUsed() + offHeapExecutionMemoryPool.memoryUsed());
}
}
/**
* Returns the execution memory consumption, in bytes, for the given task.
*/
public long getExecutionMemoryUsageForConnection(long connAttemptId) {
synchronized (this) {
assert (connAttemptId >= 0);
return (onHeapExecutionMemoryPool.getMemoryUsageConnection(connAttemptId) +
offHeapExecutionMemoryPool.getMemoryUsageConnection(connAttemptId));
}
}
/**
* Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using
* sun.misc.Unsafe.
*/
public final MemoryMode tungstenMemoryMode() {
if (conf.getBoolean("server.memory.offHeap.enabled", false)) {
assert (conf.getSizeAsBytes("server.memory.offHeap.size", 0) > 0);
assert (Platform.unaligned());
return MemoryMode.OFF_HEAP;
} else {
return MemoryMode.ON_HEAP;
}
}
/**
* The default page size, in bytes.
* <p>
* If user didn't explicitly set "server.buffer.pageSize", we figure out the default value
* by looking at the number of cores available to the process, and the total amount of memory,
* and then divide it by a factor of safety.
*/
public long pageSizeBytes() {
long minPageSize = 1L * 1024 * 1024; // 1MB
long maxPageSize = 64L * minPageSize; // 64MB
int cores = 0;
if (numCores > 0) {
cores = numCores;
} else {
cores = Runtime.getRuntime().availableProcessors();
}
// Because of rounding to next power of 2, we may have safetyFactor as 8 in worst case
int safetyFactor = 16;
long maxTungstenMemory = 0L;
MemoryMode i = tungstenMemoryMode();
if (i == MemoryMode.ON_HEAP) {
synchronized (this) {
maxTungstenMemory = onHeapExecutionMemoryPool.poolSize();
}
} else if (i == MemoryMode.OFF_HEAP) {
synchronized (this) {
maxTungstenMemory = offHeapExecutionMemoryPool.poolSize();
}
}
long size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor);
long defaultSize = Math.min(maxPageSize, Math.max(minPageSize, size));
defaultSize = conf.getSizeAsBytes("server.buffer.pageSize", defaultSize);
return defaultSize;
}
/**
* Allocates memory for use by Unsafe/Tungsten code.
*/
public final MemoryAllocator tungstenMemoryAllocator() {
MemoryMode i = tungstenMemoryMode();
if (i == MemoryMode.ON_HEAP) {
return MemoryAllocator.HEAP;
} else if (i == MemoryMode.OFF_HEAP) {
return MemoryAllocator.UNSAFE;
}
return null;
}
/**
* Get Direct Memory Usage.
*/
public final ConcurrentMap<Long, Long> getDirectMemoryUsage() {
synchronized (this) {
return offHeapExecutionMemoryPool.getMemoryForConnection();
}
}
}

View File

@@ -1,23 +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 com.actiontech.dble.memory.unsafe.memory.mm;
public enum MemoryMode {
ON_HEAP,
OFF_HEAP
}

View File

@@ -1,73 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory.mm;
import javax.annotation.concurrent.GuardedBy;
/**
* Manages bookkeeping for an adjustable-sized region of memory. This class is internal to
* the [[MemoryManager]]. See subclasses for more details.
*/
public abstract class MemoryPool {
/**
* lock [[MemoryManager]] instance, used for synchronization. We purposely erase the type
* to `Object` to avoid programming errors, since this object should only be used for
* synchronization purposes.
*/
protected final Object lock;
public MemoryPool(Object lock) {
this.lock = lock;
}
@GuardedBy("lock")
private long poolSize = 0;
/**
* Returns the current size of the pool, in bytes.
*/
public final long poolSize() {
synchronized (lock) {
return poolSize;
}
}
/**
* Returns the amount of free memory in the pool, in bytes.
*/
public long memoryFree() {
synchronized (lock) {
return (poolSize - memoryUsed());
}
}
/**
* Expands the pool by `delta` bytes.
*/
public final void incrementPoolSize(long delta) {
assert (delta >= 0);
synchronized (lock) {
poolSize += delta;
}
}
/**
* Shrinks the pool by `delta` bytes.
*/
public final void decrementPoolSize(long delta) {
synchronized (lock) {
assert (delta >= 0);
assert (delta <= poolSize);
assert (poolSize - delta >= memoryUsed());
poolSize -= delta;
}
}
/**
* Returns the amount of used memory in this pool (in bytes).
*/
protected abstract long memoryUsed();
}

View File

@@ -1,30 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory.mm;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
/**
* Created by zagnix on 2016/6/7.
*/
public class ResultMergeMemoryManager extends MemoryManager {
public ResultMergeMemoryManager(ServerPropertyConf conf, int numCores, long onHeapExecutionMemory) {
super(conf, numCores, onHeapExecutionMemory);
}
@Override
protected synchronized long acquireExecutionMemory(long numBytes, long taskAttemptId, MemoryMode memoryMode) throws InterruptedException {
if (memoryMode == MemoryMode.ON_HEAP) {
return onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId);
} else if (memoryMode == MemoryMode.OFF_HEAP) {
return offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId);
}
return 0L;
}
}

View File

@@ -1,175 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory.mm;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* Created by zagnix on 2016/6/6.
*/
public class ResultSetMemoryPool extends MemoryPool {
private static final Logger LOG = LoggerFactory.getLogger(ResultSetMemoryPool.class);
private MemoryMode memoryMode;
/**
* @param lock a [[MemoryManager]] instance to synchronize on
* @param memoryMode the type of memory tracked by this pool (on- or off-heap)
*/
public ResultSetMemoryPool(Object lock, MemoryMode memoryMode) {
super(lock);
this.memoryMode = memoryMode;
}
private String poolName() {
if (memoryMode == MemoryMode.ON_HEAP) {
return "on-heap memory";
} else if (memoryMode == MemoryMode.OFF_HEAP) {
return "off-heap memory";
}
return "off-heap memory";
}
public ConcurrentMap<Long, Long> getMemoryForConnection() {
return memoryForConnection;
}
/**
* Map from taskAttemptId -> memory consumption in bytes
*/
private ConcurrentMap<Long, Long> memoryForConnection = new ConcurrentHashMap<>();
@Override
protected long memoryUsed() {
synchronized (lock) {
long used = 0;
for (Map.Entry<Long, Long> entry : memoryForConnection.entrySet()) {
used += entry.getValue();
}
return used;
}
}
/**
* Returns the memory consumption, in bytes, for the given task.
*/
public long getMemoryUsageConnection(long taskAttemptId) {
synchronized (lock) {
if (!memoryForConnection.containsKey(taskAttemptId)) {
memoryForConnection.put(taskAttemptId, 0L);
}
return memoryForConnection.get(taskAttemptId);
}
}
/**
* Try to acquire up to `numBytes` of memory for the given task and return the number of bytes
* obtained, or 0 if none can be allocated.
* <p>
* This call may block until there is enough free memory in some situations, to make sure each
* task has a chance to ramp up to at least 1 / 8N of the total memory pool (where N is the # of
* active tasks) before it is forced to spill. This can happen if the number of tasks increase
* but an older task had a lot of memory already.
*
* @param numBytes number of bytes to acquire
* @param connAttemptId the task attempt acquiring memory
* @return the number of bytes granted to the task.
*/
public long acquireMemory(long numBytes, long connAttemptId) throws InterruptedException {
synchronized (lock) {
assert (numBytes > 0);
// Add this connection to the taskMemory map just so we can keep an accurate count of the number
// of active tasks, to let other tasks ramp down their memory in calls to `acquireMemory`
if (!memoryForConnection.containsKey(connAttemptId)) {
memoryForConnection.put(connAttemptId, 0L);
// This will later cause waiting tasks to wake up and check numTasks again
lock.notifyAll();
}
while (true) {
long numActiveCons = memoryForConnection.size();
long curMem = memoryForConnection.get(connAttemptId);
long maxPoolSize = poolSize();
long maxMemoryPerTask = maxPoolSize / numActiveCons;
long minMemoryPerTask = poolSize() / (8 * numActiveCons);
// How much we can grant this connection; keep its share within 0 <= X <= 1 / numActiveConns
long maxToGrant = Math.min(numBytes, Math.max(0, maxMemoryPerTask - curMem));
// Only give it as much memory as is free, which might be none if it reached 1 / numActiveConns
long toGrant = Math.min(maxToGrant, memoryFree());
// We want to let each connection get at least 1 / (8 * numActiveConns) before blocking;
// if we can't give it this much now, wait for other tasks to free up memory
// (this happens if older tasks allocated lots of memory before N grew)
if (toGrant < numBytes && curMem + toGrant < minMemoryPerTask) {
LOG.info("Thread " + connAttemptId + " waiting for at least 1/8N of " + poolName() + " pool to be free");
lock.wait();
} else {
long temp = memoryForConnection.get(connAttemptId);
memoryForConnection.put(connAttemptId, (temp + toGrant));
return toGrant;
}
}
}
}
/**
* Release `numBytes` of memory acquired by the given task.
*/
public void releaseMemory(long numBytes, long connAttemptId) {
synchronized (lock) {
long curMem = memoryForConnection.get(connAttemptId);
long memoryToFree = 0L;
if (curMem < numBytes) {
LOG.warn(
"Internal error: release called on $numBytes bytes but task only has $curMem bytes " +
"of memory from the " + poolName() + " pool");
memoryToFree = curMem;
} else {
memoryToFree = numBytes;
}
if (memoryForConnection.containsKey(connAttemptId)) {
long temp = memoryForConnection.get(connAttemptId);
memoryForConnection.put(connAttemptId, (temp - memoryToFree));
if (memoryForConnection.get(connAttemptId) <= 0) {
memoryForConnection.remove(connAttemptId);
}
}
// Notify waiters in acquireMemory() that memory has been freed
lock.notifyAll();
}
}
/**
* Release all memory for the given task and mark it as inactive (e.g. when a task ends).
*
* @return the number of bytes freed.
*/
public long releaseAllMemoryForeConnection(long connAttemptId) {
synchronized (lock) {
long numBytesToFree = getMemoryUsageConnection(connAttemptId);
releaseMemory(numBytesToFree, connAttemptId);
return numBytesToFree;
}
}
}

View File

@@ -1,103 +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 com.actiontech.dble.memory.unsafe.row;
import com.actiontech.dble.memory.unsafe.Platform;
/**
* A helper class to manage the data buffer for an unsafe row. The data buffer can grow and
* automatically re-point the unsafe row to it.
* <p>
* This class can be used to build a one-pass unsafe row writing program, i.e. data will be written
* to the data buffer directly and no extra copy is needed. There should be only one instance of
* this class per writing program, so that the memory segment/data buffer can be reused. Note that
* for each incoming record, we should call `reset` of BufferHolder instance before write the record
* and reuse the data buffer.
* <p>
* Generally we should call `UnsafeRow.setTotalSize` and pass in `BufferHolder.totalSize` to update
* the size of the result row, after writing a record to the buffer. However, we can skip this step
* if the fields of row are all fixed-length, as the size of result row is also fixed.
*/
public class BufferHolder {
private byte[] buffer;
private int cursor = Platform.BYTE_ARRAY_OFFSET;
private final UnsafeRow row;
private final int fixedSize;
public BufferHolder(UnsafeRow row) {
this(row, 64);
}
public BufferHolder(UnsafeRow row, int initialSize) {
this.fixedSize = UnsafeRow.calculateBitSetWidthInBytes(row.numFields()) + 8 * row.numFields();
this.buffer = new byte[fixedSize + initialSize];
this.row = row;
this.row.pointTo(buffer, buffer.length);
}
/**
* Grows the buffer by at least neededSize and points the row to the buffer.
*/
public void grow(int neededSize) {
final int length = totalSize() + neededSize;
if (buffer.length < length) {
// This will not happen frequently, because the buffer is re-used.
final byte[] tmp = new byte[length * 2];
Platform.copyMemory(
buffer,
Platform.BYTE_ARRAY_OFFSET,
tmp,
Platform.BYTE_ARRAY_OFFSET,
totalSize());
buffer = tmp;
row.pointTo(buffer, buffer.length);
}
}
public UnsafeRow getRow() {
return row;
}
public void reset() {
cursor = Platform.BYTE_ARRAY_OFFSET + fixedSize;
}
public int totalSize() {
return cursor - Platform.BYTE_ARRAY_OFFSET;
}
public byte[] getBuffer() {
return buffer;
}
public void setBuffer(byte[] buffer) {
this.buffer = buffer;
}
public int getCursor() {
return cursor;
}
public void setCursor(int cursor) {
this.cursor = cursor;
}
}

View File

@@ -1,49 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.row;
import com.actiontech.dble.sqlengine.mpp.ColMeta;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import javax.annotation.Nonnull;
import java.util.Map;
/**
* Created by zagnix on 2016/6/6.
*/
public class StructType {
private final Map<String, ColMeta> columnToIndex;
private final int fieldCount;
private OrderCol[] orderCols = null;
public StructType(@Nonnull Map<String, ColMeta> columnToIndex, int fieldCount) {
assert fieldCount >= 0;
this.columnToIndex = columnToIndex;
this.fieldCount = fieldCount;
}
public int length() {
return fieldCount;
}
public Map<String, ColMeta> getColumnToIndex() {
return columnToIndex;
}
public OrderCol[] getOrderCols() {
return orderCols;
}
public void setOrderCols(OrderCol[] orderCols) {
this.orderCols = orderCols;
}
public long apply(int i) {
return 0;
}
}

View File

@@ -1,566 +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 com.actiontech.dble.memory.unsafe.row;
import com.actiontech.dble.backend.mysql.BufferUtil;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.ByteArrayMethods;
import com.actiontech.dble.memory.unsafe.bitset.BitSetMethods;
import com.actiontech.dble.memory.unsafe.hash.Murmur3OfX86And32Bit;
import com.actiontech.dble.memory.unsafe.types.UTF8String;
import com.actiontech.dble.net.FrontendConnection;
import com.actiontech.dble.net.mysql.MySQLPacket;
import java.io.IOException;
import java.io.OutputStream;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
/**
* Modify by zagnix
* An Unsafe implementation of Row which is backed by raw memory instead of Java objects.
* <p>
* Each tuple has three parts: [null bit set] [values] [variable length portion]
* <p>
* The bit set is used for null tracking and is aligned to 8-byte word boundaries. It stores
* one bit per field.
* <p>
* In the `values` region, we store one 8-byte word per field. For fields that hold fixed-length
* primitive types, such as long, double, or int, we store the value directly in the word. For
* fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the
* base address of the row) that points to the beginning of the variable-length field, and length
* (they are combined into a long).
* <p>
* Instances of `UnsafeRow` act as pointers to row data stored in this format.
*/
public final class UnsafeRow extends MySQLPacket {
//////////////////////////////////////////////////////////////////////////////
// Static methods
//////////////////////////////////////////////////////////////////////////////
public static int calculateBitSetWidthInBytes(int numFields) {
return ((numFields + 63) / 64) * 8;
}
public static int calculateFixedPortionByteSize(int numFields) {
return 8 * numFields + calculateBitSetWidthInBytes(numFields);
}
//////////////////////////////////////////////////////////////////////////////
// Private fields and methods
//////////////////////////////////////////////////////////////////////////////
private Object baseObject;
private long baseOffset;
/**
* The number of fields in this row, used for calculating the bitset width (and in assertions)
*/
private int numFields;
/**
* The size of this row's backing data, in bytes)
*/
private int sizeInBytes;
/**
* The width of the null tracking bit set, in bytes
*/
private int bitSetWidthInBytes;
private long getFieldOffset(int ordinal) {
return baseOffset + bitSetWidthInBytes + ordinal * 8L;
}
private void assertIndexIsValid(int index) {
assert index >= 0 : "index (" + index + ") should >= 0";
assert index < numFields : "index (" + index + ") should < " + numFields;
}
//////////////////////////////////////////////////////////////////////////////
// Public methods
//////////////////////////////////////////////////////////////////////////////
/**
* Construct a new UnsafeRow. The resulting row won't be usable until `pointTo()` has been called,
* since the value returned by this constructor is equivalent to a null pointer.
*
* @param numFields the number of fields in this row
*/
public UnsafeRow(int numFields) {
this.numFields = numFields;
this.bitSetWidthInBytes = calculateBitSetWidthInBytes(numFields);
}
// for serializer
public UnsafeRow() {
}
public Object getBaseObject() {
return baseObject;
}
public long getBaseOffset() {
return baseOffset;
}
public int getSizeInBytes() {
return sizeInBytes;
}
public int numFields() {
return numFields;
}
/**
* Update this UnsafeRow to point to different backing data.
*
* @param object the base object
* @param offset the offset within the base object
* @param size the size of this row's backing data, in bytes
*/
public void pointTo(Object object, long offset, int size) {
assert numFields >= 0 : "numFields (" + numFields + ") should >= 0";
this.baseObject = object;
this.baseOffset = offset;
this.sizeInBytes = size;
}
/**
* Update this UnsafeRow to point to the underlying byte array.
*
* @param buf byte array to point to
* @param size the number of bytes valid in the byte array
*/
public void pointTo(byte[] buf, int size) {
pointTo(buf, Platform.BYTE_ARRAY_OFFSET, size);
}
public void setTotalSize(int size) {
this.sizeInBytes = size;
}
public void setNotNullAt(int i) {
assertIndexIsValid(i);
BitSetMethods.unset(baseObject, baseOffset, i);
}
public void setNullAt(int i) {
assertIndexIsValid(i);
BitSetMethods.set(baseObject, baseOffset, i);
// To preserve row equality, zero out the value when setting the column to null.
// Since this row does does not currently support updates to variable-length values, we don't
// have to worry about zeroing out that data.
Platform.putLong(baseObject, getFieldOffset(i), 0);
}
public void update(int ordinal, Object value) {
throw new UnsupportedOperationException();
}
public void setInt(int ordinal, int value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
Platform.putInt(baseObject, getFieldOffset(ordinal), value);
}
public void setLong(int ordinal, long value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
Platform.putLong(baseObject, getFieldOffset(ordinal), value);
}
public void setDouble(int ordinal, double value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
if (Double.isNaN(value)) {
value = Double.NaN;
}
Platform.putDouble(baseObject, getFieldOffset(ordinal), value);
}
public void setBoolean(int ordinal, boolean value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
Platform.putBoolean(baseObject, getFieldOffset(ordinal), value);
}
public void setShort(int ordinal, short value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
Platform.putShort(baseObject, getFieldOffset(ordinal), value);
}
public void setByte(int ordinal, byte value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
Platform.putByte(baseObject, getFieldOffset(ordinal), value);
}
public void setFloat(int ordinal, float value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
if (Float.isNaN(value)) {
value = Float.NaN;
}
Platform.putFloat(baseObject, getFieldOffset(ordinal), value);
}
public boolean isNullAt(int ordinal) {
assertIndexIsValid(ordinal);
return BitSetMethods.isSet(baseObject, baseOffset, ordinal);
}
public boolean getBoolean(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getBoolean(baseObject, getFieldOffset(ordinal));
}
public byte getByte(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getByte(baseObject, getFieldOffset(ordinal));
}
public short getShort(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getShort(baseObject, getFieldOffset(ordinal));
}
public int getInt(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getInt(baseObject, getFieldOffset(ordinal));
}
public long getLong(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getLong(baseObject, getFieldOffset(ordinal));
}
public float getFloat(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getFloat(baseObject, getFieldOffset(ordinal));
}
public double getDouble(int ordinal) {
assertIndexIsValid(ordinal);
return Platform.getDouble(baseObject, getFieldOffset(ordinal));
}
public UTF8String getUTF8String(int ordinal) {
if (isNullAt(ordinal)) return null;
final long offsetAndSize = getLong(ordinal);
final int offset = (int) (offsetAndSize >> 32);
final int size = (int) offsetAndSize;
return UTF8String.fromAddress(baseObject, baseOffset + offset, size);
}
public byte[] getBinary(int ordinal) {
if (isNullAt(ordinal)) {
return null;
} else {
final long offsetAndSize = getLong(ordinal);
final int offset = (int) (offsetAndSize >> 32);
final int size = (int) offsetAndSize;
final byte[] bytes = new byte[size];
Platform.copyMemory(
baseObject,
baseOffset + offset,
bytes,
Platform.BYTE_ARRAY_OFFSET,
size
);
return bytes;
}
}
/**
* Copies this row, returning a self-contained UnsafeRow that stores its data in an internal
* byte array rather than referencing data stored in a data page.
*/
public UnsafeRow copy() {
UnsafeRow rowCopy = new UnsafeRow(numFields);
final byte[] rowDataCopy = new byte[sizeInBytes];
Platform.copyMemory(
baseObject,
baseOffset,
rowDataCopy,
Platform.BYTE_ARRAY_OFFSET,
sizeInBytes
);
rowCopy.pointTo(rowDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes);
return rowCopy;
}
/**
* Creates an empty UnsafeRow from a byte array with specified numBytes and numFields.
* The returned row is invalid until we call copyFrom on it.
*/
public static UnsafeRow createFromByteArray(int numBytes, int numFields) {
final UnsafeRow row = new UnsafeRow(numFields);
row.pointTo(new byte[numBytes], numBytes);
return row;
}
/**
* Copies the input UnsafeRow to this UnsafeRow, and resize the underlying byte[] when the
* input row is larger than this row.
*/
public void copyFrom(UnsafeRow row) {
// copyFrom is only available for UnsafeRow created from byte array.
assert (baseObject instanceof byte[]) && baseOffset == Platform.BYTE_ARRAY_OFFSET;
if (row.sizeInBytes > this.sizeInBytes) {
// resize the underlying byte[] if it's not large enough.
this.baseObject = new byte[row.sizeInBytes];
}
Platform.copyMemory(
row.baseObject, row.baseOffset, this.baseObject, this.baseOffset, row.sizeInBytes);
// update the sizeInBytes.
this.sizeInBytes = row.sizeInBytes;
}
/**
* Write this UnsafeRow's underlying bytes to the given OutputStream.
*
* @param out the stream to write to.
* @param writeBuffer a byte array for buffering chunks of off-heap data while writing to the
* output stream. If this row is backed by an on-heap byte array, then this
* buffer will not be used and may be null.
*/
public void writeToStream(OutputStream out, byte[] writeBuffer) throws IOException {
if (baseObject instanceof byte[]) {
int offsetInByteArray = (int) (Platform.BYTE_ARRAY_OFFSET - baseOffset);
out.write((byte[]) baseObject, offsetInByteArray, sizeInBytes);
} else {
int dataRemaining = sizeInBytes;
long rowReadPosition = baseOffset;
while (dataRemaining > 0) {
int toTransfer = Math.min(writeBuffer.length, dataRemaining);
Platform.copyMemory(
baseObject, rowReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer);
out.write(writeBuffer, 0, toTransfer);
rowReadPosition += toTransfer;
dataRemaining -= toTransfer;
}
}
}
@Override
public int hashCode() {
return Murmur3OfX86And32Bit.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, 42);
}
@Override
public boolean equals(Object other) {
if (other instanceof UnsafeRow) {
UnsafeRow o = (UnsafeRow) other;
return (sizeInBytes == o.sizeInBytes) &&
ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset,
sizeInBytes);
}
return false;
}
/**
* Returns the underlying bytes for this UnsafeRow.
*/
public byte[] getBytes() {
if (baseObject instanceof byte[] && baseOffset == Platform.BYTE_ARRAY_OFFSET &&
(((byte[]) baseObject).length == sizeInBytes)) {
return (byte[]) baseObject;
} else {
byte[] bytes = new byte[sizeInBytes];
Platform.copyMemory(baseObject, baseOffset, bytes, Platform.BYTE_ARRAY_OFFSET, sizeInBytes);
return bytes;
}
}
public static final byte NULL_MARK = (byte) 251;
public static final byte EMPTY_MARK = (byte) 0;
@Override
public ByteBuffer write(ByteBuffer bb, FrontendConnection c,
boolean writeSocketIfFull) {
bb = c.checkWriteBuffer(bb, MySQLPacket.PACKET_HEADER_SIZE, writeSocketIfFull);
BufferUtil.writeUB3(bb, calcPacketSize());
bb.put(packetId);
for (int i = 0; i < numFields; i++) {
if (!isNullAt(i)) {
byte[] fv = this.getBinary(i);
if (fv.length == 0) {
bb = c.checkWriteBuffer(bb, 1, writeSocketIfFull);
bb.put(UnsafeRow.EMPTY_MARK);
} else {
bb = c.checkWriteBuffer(bb, BufferUtil.getLength(fv),
writeSocketIfFull);
BufferUtil.writeLength(bb, fv.length);
/**
* write data to Writer Buffer
*/
bb = c.writeToBuffer(fv, bb);
}
} else {
//Col null value
bb = c.checkWriteBuffer(bb, 1, writeSocketIfFull);
bb.put(UnsafeRow.NULL_MARK);
}
}
return bb;
}
@Override
public int calcPacketSize() {
int size = 0;
for (int i = 0; i < numFields; i++) {
byte[] v = this.getBinary(i);
size += (v == null || v.length == 0) ? 1 : BufferUtil.getLength(v);
}
return size;
}
public BigDecimal getDecimal(int ordinal, int scale) {
if (isNullAt(ordinal)) {
return null;
}
byte[] bytes = getBinary(ordinal);
BigInteger bigInteger = new BigInteger(bytes);
BigDecimal javaDecimal = new BigDecimal(bigInteger, scale);
return javaDecimal;
}
/**
* update <strong>exist</strong> decimal column value to new decimal value
* <p>
* NOTE: decimal max precision is limit to 38
*
* @param ordinal
* @param value
* @param precision
*/
public void updateDecimal(int ordinal, BigDecimal value) {
assertIndexIsValid(ordinal);
// fixed length
long cursor = getLong(ordinal) >>> 32;
assert cursor > 0 : "invalid cursor " + cursor;
// zero-out the bytes
Platform.putLong(baseObject, baseOffset + cursor, 0L);
Platform.putLong(baseObject, baseOffset + cursor + 8, 0L);
if (value == null) {
setNullAt(ordinal);
// keep the offset for future update
Platform.putLong(baseObject, getFieldOffset(ordinal), cursor << 32);
} else {
final BigInteger integer = value.unscaledValue();
byte[] bytes = integer.toByteArray();
assert (bytes.length <= 16);
// Write the bytes to the variable length portion.
Platform.copyMemory(bytes, Platform.BYTE_ARRAY_OFFSET, baseObject, baseOffset + cursor, bytes.length);
setLong(ordinal, (cursor << 32) | ((long) bytes.length));
}
}
/**
* public Decimal getDecimal(int ordinal, int precision, int scale) {
* if (isNullAt(ordinal)) {
* return null;
* }
* if (precision <= Decimal.MAX_LONG_DIGITS()) {
* return Decimal.createUnsafe(getLong(ordinal), precision, scale);
* } else {
* byte[] bytes = getBinary(ordinal);
* BigInteger bigInteger = new BigInteger(bytes);
* BigDecimal javaDecimal = new BigDecimal(bigInteger, scale);
* return Decimal.apply(javaDecimal, precision, scale);
* }
* }
* <p>
* public void setDecimal(int ordinal, Decimal value, int precision) {
* assertIndexIsValid(ordinal);
* if (precision <= Decimal.MAX_LONG_DIGITS()) {
* // compact format
* if (value == null) {
* setNullAt(ordinal);
* } else {
* setLong(ordinal, value.toUnscaledLong());
* }
* } else {
* // fixed length
* long cursor = getLong(ordinal) >>> 32;
* assert cursor > 0 : "invalid cursor " + cursor;
* // zero-out the bytes
* Platform.putLong(baseObject, baseOffset + cursor, 0L);
* Platform.putLong(baseObject, baseOffset + cursor + 8, 0L);
* <p>
* if (value == null) {
* setNullAt(ordinal);
* // keep the offset for future update
* Platform.putLong(baseObject, getFieldOffset(ordinal), cursor << 32);
* } else {
* <p>
* final BigInteger integer = value.toJavaBigDecimal().unscaledValue();
* byte[] bytes = integer.toByteArray();
* assert(bytes.length <= 16);
* <p>
* // Write the bytes to the variable length portion.
* Platform.copyMemory(
* bytes, Platform.BYTE_ARRAY_OFFSET, baseObject, baseOffset + cursor, bytes.length);
* setLong(ordinal, (cursor << 32) | ((long) bytes.length));
* }
* }
* }
*/
@Override
protected String getPacketInfo() {
return "MySQL RowData Packet";
}
// This is for debugging
@Override
public String toString() {
StringBuilder build = new StringBuilder("[");
for (int i = 0; i < sizeInBytes; i += 8) {
if (i != 0) build.append(',');
build.append(Long.toHexString(Platform.getLong(baseObject, baseOffset + i)));
}
build.append(']');
return build.toString();
}
public boolean anyNull() {
return BitSetMethods.anySet(baseObject, baseOffset, bitSetWidthInBytes / 8);
}
}

View File

@@ -1,233 +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 com.actiontech.dble.memory.unsafe.row;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.ByteArrayMethods;
import com.actiontech.dble.memory.unsafe.bitset.BitSetMethods;
import java.math.BigDecimal;
/**
* A helper class to write data into global row buffer using `UnsafeRow` format.
* <p>
* It will remember the offset of row buffer which it starts to write, and move the cursor of row
* buffer while writing. If new data(can be the input record if this is the outermost writer, or
* nested struct if this is an inner writer) comes, the starting cursor of row buffer may be
* changed, so we need to call `UnsafeRowWriter.reset` before writing, to update the
* `startingOffset` and clear out null bits.
* <p>
* Note that if this is the outermost writer, which means we will always write from the very
* beginning of the global row buffer, we don't need to update `startingOffset` and can just call
* `zeroOutNullBytes` before writing new data.
*/
public class UnsafeRowWriter {
private final BufferHolder holder;
// The offset of the global buffer where we start to write this row.
private int startingOffset;
private final int nullBitsSize;
private final int fixedSize;
public UnsafeRowWriter(BufferHolder holder, int numFields) {
this.holder = holder;
this.nullBitsSize = UnsafeRow.calculateBitSetWidthInBytes(numFields);
this.fixedSize = nullBitsSize + 8 * numFields;
this.startingOffset = holder.getCursor();
}
/**
* Resets the `startingOffset` according to the current cursor of row buffer, and clear out null
* bits. This should be called before we write a new nested struct to the row buffer.
*/
public void reset() {
this.startingOffset = holder.getCursor();
// grow the global buffer to make sure it has enough space to write fixed-length data.
holder.grow(fixedSize);
holder.setCursor(holder.getCursor() + fixedSize);
zeroOutNullBytes();
}
/**
* Clears out null bits. This should be called before we write a new row to row buffer.
*/
public void zeroOutNullBytes() {
for (int i = 0; i < nullBitsSize; i += 8) {
Platform.putLong(holder.getBuffer(), startingOffset + i, 0L);
}
}
private void zeroOutPaddingBytes(int numBytes) {
if ((numBytes & 0x07) > 0) {
Platform.putLong(holder.getBuffer(), holder.getCursor() + ((numBytes >> 3) << 3), 0L);
}
}
public BufferHolder holder() {
return holder;
}
public boolean isNullAt(int ordinal) {
return BitSetMethods.isSet(holder.getBuffer(), startingOffset, ordinal);
}
public void setNullAt(int ordinal) {
BitSetMethods.set(holder.getBuffer(), startingOffset, ordinal);
Platform.putLong(holder.getBuffer(), getFieldOffset(ordinal), 0L);
}
public long getFieldOffset(int ordinal) {
return startingOffset + nullBitsSize + 8 * ordinal;
}
public void setOffsetAndSize(int ordinal, long size) {
setOffsetAndSize(ordinal, holder.getCursor(), size);
}
public void setOffsetAndSize(int ordinal, long currentCursor, long size) {
final long relativeOffset = currentCursor - startingOffset;
final long fieldOffset = getFieldOffset(ordinal);
final long offsetAndSize = (relativeOffset << 32) | size;
Platform.putLong(holder.getBuffer(), fieldOffset, offsetAndSize);
}
// Do word alignment for this row and grow the row buffer if needed.
// todo: remove this after we make unsafe array data word align.
public void alignToWords(int numBytes) {
final int remainder = numBytes & 0x07;
if (remainder > 0) {
final int paddingBytes = 8 - remainder;
holder.grow(paddingBytes);
for (int i = 0; i < paddingBytes; i++) {
Platform.putByte(holder.getBuffer(), holder.getCursor(), (byte) 0);
holder.setCursor(holder.getCursor() + 1);
}
}
}
public void write(int ordinal, boolean value) {
final long offset = getFieldOffset(ordinal);
Platform.putLong(holder.getBuffer(), offset, 0L);
Platform.putBoolean(holder.getBuffer(), offset, value);
}
public void write(int ordinal, byte value) {
final long offset = getFieldOffset(ordinal);
Platform.putLong(holder.getBuffer(), offset, 0L);
Platform.putByte(holder.getBuffer(), offset, value);
}
public void write(int ordinal, short value) {
final long offset = getFieldOffset(ordinal);
Platform.putLong(holder.getBuffer(), offset, 0L);
Platform.putShort(holder.getBuffer(), offset, value);
}
public void write(int ordinal, int value) {
final long offset = getFieldOffset(ordinal);
Platform.putLong(holder.getBuffer(), offset, 0L);
Platform.putInt(holder.getBuffer(), offset, value);
}
public void write(int ordinal, long value) {
Platform.putLong(holder.getBuffer(), getFieldOffset(ordinal), value);
}
public void write(int ordinal, float value) {
if (Float.isNaN(value)) {
value = Float.NaN;
}
final long offset = getFieldOffset(ordinal);
Platform.putLong(holder.getBuffer(), offset, 0L);
Platform.putFloat(holder.getBuffer(), offset, value);
}
public void write(int ordinal, double value) {
if (Double.isNaN(value)) {
value = Double.NaN;
}
Platform.putDouble(holder.getBuffer(), getFieldOffset(ordinal), value);
}
public void write(int ordinal, byte[] input) {
if (input == null) {
return;
}
write(ordinal, input, 0, input.length);
}
public void write(int ordinal, byte[] input, int offset, int numBytes) {
final int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes);
// grow the global buffer before writing data.
holder.grow(roundedSize);
zeroOutPaddingBytes(numBytes);
// Write the bytes to the variable length portion.
Platform.copyMemory(input, Platform.BYTE_ARRAY_OFFSET + offset,
holder.getBuffer(), holder.getCursor(), numBytes);
setOffsetAndSize(ordinal, numBytes);
// move the cursor forward.
holder.setCursor(holder.getCursor() + roundedSize);
}
/**
* different from Spark, we use java BigDecimal here,
* and we limit the max precision to be 38 because the bytes length limit to be 16
*
* @param ordinal
* @param input
*/
public void write(int ordinal, BigDecimal input) {
// grow the global buffer before writing data.
holder.grow(16);
// zero-out the bytes
Platform.putLong(holder.getBuffer(), holder.getCursor(), 0L);
Platform.putLong(holder.getBuffer(), holder.getCursor() + 8, 0L);
// Make sure Decimal object has the same scale as DecimalType.
// Note that we may pass in null Decimal object to set null for it.
if (input == null) {
BitSetMethods.set(holder.getBuffer(), startingOffset, ordinal);
// keep the offset for future update
setOffsetAndSize(ordinal, 0L);
} else {
final byte[] bytes = input.unscaledValue().toByteArray();
assert bytes.length <= 16;
// Write the bytes to the variable length portion.
Platform.copyMemory(bytes, Platform.BYTE_ARRAY_OFFSET, holder.getBuffer(), holder.getCursor(), bytes.length);
setOffsetAndSize(ordinal, bytes.length);
}
// move the cursor forward.
holder.setCursor(holder.getCursor() + 16);
}
}

View File

@@ -1,31 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
/**
* Created by zagnix on 2016/6/6.
*/
public abstract class ConnectionId {
protected String name;
public abstract String getBlockName();
@Override
public boolean equals(Object arg0) {
return super.equals(arg0);
}
@Override
public int hashCode() {
return super.hashCode();
}
@Override
public String toString() {
return super.toString();
}
}

View File

@@ -1,44 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import java.io.File;
import java.io.IOException;
/**
* Created by zagnix on 2016/6/3.
*/
public class DataNodeDiskManager {
private ServerPropertyConf conf;
private boolean deleteFilesOnStop;
public DataNodeDiskManager(ServerPropertyConf conf, boolean deleteFilesOnStop) {
this.conf = conf;
this.deleteFilesOnStop = deleteFilesOnStop;
}
public DataNodeFileManager diskBlockManager() throws IOException {
return new DataNodeFileManager(conf, deleteFilesOnStop);
}
/**
* A short circuited method to get a block writer that can write data directly to disk.
* The Block will be appended to the File specified by filename. Callers should handle error
* cases.
*/
public DiskRowWriter getDiskWriter(
File file,
SerializerInstance serializerInstance,
int bufferSize) throws IOException {
boolean syncWrites = conf.getBoolean("server.merge.sync", false);
return new DiskRowWriter(file, serializerInstance, bufferSize, syncWrites);
}
}

View File

@@ -1,182 +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 com.actiontech.dble.memory.unsafe.storage;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* Creates and maintains the logical mapping between logical blocks and physical on-disk
* locations. One block is mapped to one file with a name given by its BlockId.
* <p>
* Block files are hashed among the directories listed in local.dir
*/
public class DataNodeFileManager {
private static final Logger LOG = LoggerFactory.getLogger(DataNodeFileManager.class);
private boolean deleteFilesOnStop;
/**
* TODO: delete tmp file
*/
// The content of subDirs is immutable but the content of subDirs(i) is mutable. And the content
// of subDirs(i) is protected by the lock of subDirs(i)
// private val shutdownHook ;
/* Create one local directory for each path mentioned in spark.local.dir; then, inside this
* directory, create multiple subdirectories that we will hash files into, in order to avoid
* having really large inodes at the top level. */
private List<File> localDirs;
private int subDirsPerLocalDir;
private ConcurrentMap<Integer, ArrayList<File>> subDirs;
public DataNodeFileManager(ServerPropertyConf conf, boolean deleteFilesOnStop) throws IOException {
this.deleteFilesOnStop = deleteFilesOnStop;
subDirsPerLocalDir = conf.getInt("server.diskStore.subDirectories", 64);
localDirs = createLocalDirs(conf);
subDirs = new ConcurrentHashMap<>(localDirs.size());
for (int i = 0; i < localDirs.size(); i++) {
ArrayList<File> list = new ArrayList<>(subDirsPerLocalDir);
for (int j = 0; j < subDirsPerLocalDir; j++) {
list.add(i, null);
}
subDirs.put(i, list);
}
}
/**
* Produces a unique block id and File suitable for storing local intermediate results.
*/
public TempDataNodeId createTempLocalBlock() throws IOException {
TempDataNodeId blockId = new TempDataNodeId(UUID.randomUUID().toString());
while (getFile(blockId).exists()) {
blockId = new TempDataNodeId(UUID.randomUUID().toString());
}
return blockId;
}
/**
* Looks up a file by hashing it into one of our local subdirectories.
*/
// This method should be kept in sync with
// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getFile().
public File getFile(String filename) throws IOException {
// Figure out which local directory it hashes to, and which subdirectory in that
int hash = JavaUtils.nonNegativeHash(filename);
int dirId = hash % localDirs.size();
int subDirId = (hash / localDirs.size()) % subDirsPerLocalDir;
synchronized (this) {
File file = subDirs.get(dirId).get(subDirId);
if (file != null) {
LOG.warn(file.getName() + " exist !");
} else {
file = new File(localDirs.get(dirId), String.valueOf(subDirId));
if (!file.exists() && !file.mkdir()) {
throw new IOException("Failed to create local dir in $newDir.");
}
subDirs.get(dirId).add(subDirId, file);
}
}
return new File(subDirs.get(dirId).get(subDirId), filename);
}
public File getFile(ConnectionId connectionId) throws IOException {
return getFile(connectionId.name);
}
/**
* TODO config root
* Create local directories for storing block data. These directories are
* located inside configured local directories and won't
* be deleted on JVM exit when using the external shuffle service.
*/
private List<File> createLocalDirs(ServerPropertyConf conf) {
String rootDirs = conf.getString("server.local.dirs", "datanode");
String[] rootDirArray = rootDirs.split(",");
List<File> dirs = new ArrayList<>();
for (String rootDir : rootDirArray) {
try {
File localDir = JavaUtils.createDirectory(rootDir, "datenode");
dirs.add(localDir);
} catch (Exception e) {
LOG.error("Failed to create local dir in " + rootDir + ". Ignoring this directory.");
}
}
if (dirs.isEmpty()) {
throw new RuntimeException("can't createLocalDirs in " + rootDirs);
}
return dirs;
}
/**
* Cleanup local dirs and stop shuffle sender.
*/
public void stop() {
doStop();
}
private void doStop() {
if (deleteFilesOnStop) {
File localDir;
int i = 0;
if (LOG.isDebugEnabled()) {
LOG.debug("local dirs size:" + localDirs.size());
}
while (i < localDirs.size() && localDirs.size() > 0) {
localDir = localDirs.get(i);
//System.out.println(localDir);
if (localDir.isDirectory() && localDir.exists()) {
try {
JavaUtils.deleteRecursively(localDir);
} catch (Exception e) {
LOG.error(e.getMessage());
}
}
i++;
}
}
}
}

View File

@@ -1,33 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
/**
* Created by zagnix on 2016/6/3.
*/
public abstract class DeserializationStream {
/**
* The most general-purpose method to read an object.
*/
public abstract <T> T readObject();
/**
* Reads the object representing the key of a key-value pair.
*/
public <T> T readKey() {
return readObject();
}
/**
* Reads the object representing the value of a key-value pair.
*/
public <T> T readValue() {
return readObject();
}
public abstract void close();
}

View File

@@ -1,230 +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 com.actiontech.dble.memory.unsafe.storage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.nio.channels.FileChannel;
/**
* A class for writing JVM objects directly to a file on disk. This class allows data to be appended
* to an existing block and can guarantee atomicity in the case of faults as it allows the caller to
* revert partial writes.
* <p>
* This class does not support concurrent writes. Also, once the writer has been opened it cannot be
* reopened again.
*/
public class DiskRowWriter extends OutputStream {
/**
* The file channel, used for repositioning / truncating the file.
*/
private static final Logger LOG = LoggerFactory.getLogger(DiskRowWriter.class);
private FileChannel channel = null;
private OutputStream bs = null;
private FileOutputStream fos = null;
private SerializationStream objOut = null;
private boolean initialized = false;
private boolean hasBeenClosed = false;
/**
* Cursors used to represent positions in the file.
* <p>
* xxxxxxxx|--------|--- |
* ^ ^ ^
* | | finalPosition
* | reportedPosition
* initialPosition
* <p>
* initialPosition: Offset in the file where we start writing. Immutable.
* reportedPosition: Position at the time of the last update to the write metrics.
* finalPosition: Offset where we stopped writing. Set on closeAndCommit() then never changed.
* -----: Current writes to the underlying file.
* xxxxx: Existing contents of the file.
*/
private long initialPosition = 0;
private long reportedPosition = 0;
/**
* Keep track of number of records written and also use this to periodically
* output bytes written since the latter is expensive to do for each record.
*/
private long numRecordsWritten = 0;
private File file;
private SerializerInstance serializerInstance;
private int bufferSize;
private boolean syncWrites;
// These write metrics concurrently shared with other active DiskBlockObjectWriters who
// are themselves performing writes. All updates must be relative.
public DiskRowWriter(
File file,
SerializerInstance serializerInstance,
int bufferSize,
boolean syncWrites) throws IOException {
this.file = file;
this.serializerInstance = serializerInstance;
this.bufferSize = bufferSize;
this.syncWrites = syncWrites;
new FileOutputStream(file, false).close(); // for clean file
/*
ShuffleWriteMetrics writeMetrics,
*/
initialPosition = file.length();
reportedPosition = initialPosition;
}
public DiskRowWriter open() throws FileNotFoundException {
if (hasBeenClosed) {
throw new IllegalStateException("Writer already closed. Cannot be reopened.");
}
fos = new FileOutputStream(file, true);
channel = fos.getChannel();
bs = new BufferedOutputStream(fos, bufferSize);
objOut = serializerInstance.serializeStream(bs);
initialized = true;
return this;
}
@Override
public void close() {
if (initialized) {
try {
if (syncWrites) {
//Force outstanding writes to disk and track how long it takes
objOut.flush();
long start = System.nanoTime();
fos.getFD().sync();
// writeMetrics.incWriteTime(System.nanoTime() - start);
}
} catch (IOException e) {
LOG.error(e.getMessage());
} finally {
objOut.close();
}
channel = null;
bs = null;
fos = null;
objOut = null;
initialized = false;
hasBeenClosed = true;
}
}
public boolean isOpen() {
return objOut != null;
}
/**
* Flush the partial writes and commit them as a single atomic block.
*/
public void commitAndClose() throws IOException {
if (initialized) {
// NOTE: Because Kryo doesnt flush the underlying stream we explicitly flush both the
// serializer stream and the lower level stream.
objOut.flush();
bs.flush();
close();
}
}
/**
* Reverts writes that havent been flushed yet. Callers should invoke this function
* when there are runtime exceptions. This method will not throw, though it may be
* unsuccessful in truncating written data.
*
* @return the file that this DiskRowWriter wrote to.
*/
public File revertPartialWritesAndClose() throws IOException {
// Discard current writes. We do this by flushing the outstanding writes and then
// truncating the file to its initial position.
try {
if (initialized) {
// writeMetrics.decBytesWritten(reportedPosition - initialPosition)
// writeMetrics.decRecordsWritten(numRecordsWritten)
objOut.flush();
bs.flush();
close();
}
try (FileOutputStream truncateStream = new FileOutputStream(file, true)) {
truncateStream.getChannel().truncate(initialPosition);
return file;
}
} catch (Exception e) {
LOG.error(e.getMessage());
return file;
}
}
@Override
public void write(int b) {
throw new UnsupportedOperationException();
}
@Override
public void write(byte[] kvBytes, int offs, int len) throws IOException {
if (!initialized) {
open();
}
bs.write(kvBytes, offs, len);
}
/**
* Notify the writer that a record worth of bytes has been written with OutputStream#write.
*/
public void recordWritten() throws IOException {
numRecordsWritten += 1;
//writeMetrics.incRecordsWritten(1)
//TODO: call updateBytesWritten() less frequently.
if (numRecordsWritten % 32 == 0) {
updateBytesWritten();
}
}
/**
* Report the number of bytes written in this writers shuffle write metrics.
* Note that this is only valid before the underlying streams are closed.
*/
private void updateBytesWritten() throws IOException {
long pos = channel.position();
//writeMetrics.incBytesWritten(pos - reportedPosition)
reportedPosition = pos;
}
@Override
public void flush() throws IOException {
objOut.flush();
bs.flush();
}
}

View File

@@ -1,90 +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 com.actiontech.dble.memory.unsafe.storage;
import com.actiontech.dble.memory.unsafe.Platform;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
/**
* Unfortunately, we need a serializer instance in order to construct a DiskRowWriter.
* Our shuffle write path doesn't actually use this serializer (since we end up calling the
* `write() OutputStream methods), but DiskRowWriter still calls some methods on it. To work
* around this, we pass a dummy no-op serializer.
*/
public final class DummySerializerInstance extends SerializerInstance {
public static final DummySerializerInstance INSTANCE = new DummySerializerInstance();
private DummySerializerInstance() {
}
@Override
public SerializationStream serializeStream(final OutputStream s) {
return new SerializationStream() {
@Override
public SerializationStream writeObject(Object o) {
return null;
}
@Override
public void flush() {
// Need to implement this because DiskObjectWriter uses it to flush the compression stream
try {
s.flush();
} catch (IOException e) {
Platform.throwException(e);
}
}
@Override
public void close() {
// Need to implement this because DiskObjectWriter uses it to close the compression stream
try {
s.close();
} catch (IOException e) {
Platform.throwException(e);
}
}
};
}
public <T> ByteBuffer serialize(T t, T ev1) {
throw new UnsupportedOperationException();
}
public DeserializationStream deserializeStream(InputStream s) {
throw new UnsupportedOperationException();
}
public <T> T deserialize(ByteBuffer bytes, ClassLoader loader, T ev1) {
throw new UnsupportedOperationException();
}
public <T> T deserialize(ByteBuffer bytes, T ev1) {
throw new UnsupportedOperationException();
}
}

View File

@@ -1,35 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
/**
* Created by zagnix on 2016/6/3.
*/
public abstract class SerializationStream {
/**
* The most general-purpose method to write an object.
*/
public abstract <T> SerializationStream writeObject(T t);
/**
* Writes the object representing the key of a key-value pair.
*/
public <T> SerializationStream writeKey(T key) {
return writeObject(key);
}
/**
* Writes the object representing the value of a key-value pair.
*/
public <T> SerializationStream writeValue(T value) {
return writeObject(value);
}
public abstract void flush();
public abstract void close();
}

View File

@@ -1,18 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
import java.io.InputStream;
import java.io.OutputStream;
/**
* Created by zagnix on 2016/6/3.
*/
public abstract class SerializerInstance {
protected abstract SerializationStream serializeStream(OutputStream s);
protected abstract DeserializationStream deserializeStream(InputStream s);
}

View File

@@ -1,30 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
import java.io.InputStream;
import java.io.OutputStream;
/**
* Created by zagnix on 2016/6/3.
*/
public class SerializerManager {
/**
* Wrap an output stream for compression if block compression is enabled for its block type
*/
public OutputStream wrapForCompression(ConnectionId blockId, OutputStream s) {
return s;
}
/**
* Wrap an input stream for compression if block compression is enabled for its block type
*/
public InputStream wrapForCompression(ConnectionId blockId, InputStream s) {
return s;
}
}

View File

@@ -1,25 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
/**
* Created by zagnix on 2016/6/3.
*/
public class TempDataNodeId extends ConnectionId {
private String uuid;
public TempDataNodeId(String uuid) {
super();
this.name = uuid;
this.uuid = uuid;
}
@Override
public String getBlockName() {
return "temp_local_" + uuid;
}
}

View File

@@ -1,78 +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 com.actiontech.dble.memory.unsafe.types;
import com.actiontech.dble.memory.unsafe.Platform;
import java.util.Arrays;
public final class ByteArray {
private ByteArray() {
}
public static final byte[] EMPTY_BYTE = new byte[0];
/**
* Writes the content of a byte array into a memory address, identified by an object and an
* offset. The target memory address must already been allocated, and have enough space to
* hold all the bytes in this string.
*/
public static void writeToMemory(byte[] src, Object target, long targetOffset) {
Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET, target, targetOffset, src.length);
}
/**
* Returns a 64-bit integer that can be used as the prefix used in sorting.
*/
public static long getPrefix(byte[] bytes) {
if (bytes == null) {
return 0L;
} else {
final int minLen = Math.min(bytes.length, 8);
long p = 0;
for (int i = 0; i < minLen; ++i) {
p |= (128L + Platform.getByte(bytes, Platform.BYTE_ARRAY_OFFSET + i)) << (56 - 8 * i);
}
return p;
}
}
public static byte[] subStringSQL(byte[] bytes, int pos, int len) {
// This pos calculation is according to UTF8String#subStringSQL
if (pos > bytes.length) {
return EMPTY_BYTE;
}
int start = 0;
int end;
if (pos > 0) {
start = pos - 1;
} else if (pos < 0) {
start = bytes.length + pos;
}
if ((bytes.length - start) < len) {
end = bytes.length;
} else {
end = start + len;
}
start = Math.max(start, 0); // underflow
if (start >= end) {
return EMPTY_BYTE;
}
return Arrays.copyOfRange(bytes, start, end);
}
}

View File

@@ -1,338 +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 com.actiontech.dble.memory.unsafe.types;
import java.io.Serializable;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* The internal representation of interval type.
*/
public final class CalendarInterval implements Serializable {
public static final long MICROS_PER_MILLI = 1000L;
public static final long MICROS_PER_SECOND = MICROS_PER_MILLI * 1000;
public static final long MICROS_PER_MINUTE = MICROS_PER_SECOND * 60;
public static final long MICROS_PER_HOUR = MICROS_PER_MINUTE * 60;
public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24;
public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7;
/**
* A function to generate regex which matches interval string's unit part like "3 years".
* <p>
* First, we can leave out some units in interval string, and we only care about the value of
* unit, so here we use non-capturing group to wrap the actual regex.
* At the beginning of the actual regex, we should match spaces before the unit part.
* Next is the number part, starts with an optional "-" to represent negative value. We use
* capturing group to wrap this part as we need the value later.
* Finally is the unit name, ends with an optional "s".
*/
private static String unitRegex(String unit) {
return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?";
}
private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") +
unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") +
unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond"));
private static Pattern yearMonthPattern =
Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$");
private static Pattern dayTimePattern =
Pattern.compile("^(?:['|\"])?([+|-])?(\\d+) (\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$");
private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$");
private static long toLong(String s) {
if (s == null) {
return 0;
} else {
return Long.parseLong(s);
}
}
public static CalendarInterval fromString(String s) {
if (s == null) {
return null;
}
s = s.trim();
Matcher m = p.matcher(s);
if (!m.matches() || s.equals("interval")) {
return null;
} else {
long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK;
microseconds += toLong(m.group(4)) * MICROS_PER_DAY;
microseconds += toLong(m.group(5)) * MICROS_PER_HOUR;
microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE;
microseconds += toLong(m.group(7)) * MICROS_PER_SECOND;
microseconds += toLong(m.group(8)) * MICROS_PER_MILLI;
microseconds += toLong(m.group(9));
long months = toLong(m.group(1)) * 12 + toLong(m.group(2));
return new CalendarInterval((int) months, microseconds);
}
}
public static long toLongWithRange(String fieldName,
String s, long minValue, long maxValue) throws IllegalArgumentException {
long result = 0;
if (s != null) {
result = Long.parseLong(s);
if (result < minValue || result > maxValue) {
throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]",
fieldName, result, minValue, maxValue));
}
}
return result;
}
/**
* Parse YearMonth string in form: [-]YYYY-MM
* <p>
* adapted from HiveIntervalYearMonth.valueOf
*/
public static CalendarInterval fromYearMonthString(String s) throws IllegalArgumentException {
CalendarInterval result = null;
if (s == null) {
throw new IllegalArgumentException("Interval year-month string was null");
}
s = s.trim();
Matcher m = yearMonthPattern.matcher(s);
if (!m.matches()) {
throw new IllegalArgumentException(
"Interval string does not match year-month format of 'y-m': " + s);
} else {
try {
int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1;
int years = (int) toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE);
int months = (int) toLongWithRange("month", m.group(3), 0, 11);
result = new CalendarInterval(sign * (years * 12 + months), 0);
} catch (Exception e) {
throw new IllegalArgumentException(
"Error parsing interval year-month string: " + e.getMessage(), e);
}
}
return result;
}
/**
* Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn
* <p>
* adapted from HiveIntervalDayTime.valueOf
*/
public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException {
CalendarInterval result = null;
if (s == null) {
throw new IllegalArgumentException("Interval day-time string was null");
}
s = s.trim();
Matcher m = dayTimePattern.matcher(s);
if (!m.matches()) {
throw new IllegalArgumentException(
"Interval string does not match day-time format of 'd h:m:s.n': " + s);
} else {
try {
int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1;
long days = toLongWithRange("day", m.group(2), 0, Integer.MAX_VALUE);
long hours = toLongWithRange("hour", m.group(3), 0, 23);
long minutes = toLongWithRange("minute", m.group(4), 0, 59);
long seconds = toLongWithRange("second", m.group(5), 0, 59);
// Hive allow nanosecond precision interval
long nanos = toLongWithRange("nanosecond", m.group(7), 0L, 999999999L);
result = new CalendarInterval(0, sign * (
days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE +
seconds * MICROS_PER_SECOND + nanos / 1000L));
} catch (Exception e) {
throw new IllegalArgumentException(
"Error parsing interval day-time string: " + e.getMessage(), e);
}
}
return result;
}
public static CalendarInterval fromSingleUnitString(String unit, String s)
throws IllegalArgumentException {
CalendarInterval result = null;
if (s == null) {
throw new IllegalArgumentException(String.format("Interval %s string was null", unit));
}
s = s.trim();
Matcher m = quoteTrimPattern.matcher(s);
if (!m.matches()) {
throw new IllegalArgumentException(
"Interval string does not match day-time format of 'd h:m:s.n': " + s);
} else {
try {
switch (unit) {
case "year":
int year = (int) toLongWithRange("year", m.group(1),
Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12);
result = new CalendarInterval(year * 12, 0L);
break;
case "month":
int month = (int) toLongWithRange("month", m.group(1),
Integer.MIN_VALUE, Integer.MAX_VALUE);
result = new CalendarInterval(month, 0L);
break;
case "week":
long week = toLongWithRange("week", m.group(1),
Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK);
result = new CalendarInterval(0, week * MICROS_PER_WEEK);
break;
case "day":
long day = toLongWithRange("day", m.group(1),
Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY);
result = new CalendarInterval(0, day * MICROS_PER_DAY);
break;
case "hour":
long hour = toLongWithRange("hour", m.group(1),
Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR);
result = new CalendarInterval(0, hour * MICROS_PER_HOUR);
break;
case "minute":
long minute = toLongWithRange("minute", m.group(1),
Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE);
result = new CalendarInterval(0, minute * MICROS_PER_MINUTE);
break;
case "second": {
long micros = parseSecondNano(m.group(1));
result = new CalendarInterval(0, micros);
break;
}
case "millisecond":
long millisecond = toLongWithRange("millisecond", m.group(1),
Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI);
result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI);
break;
case "microsecond": {
long micros = Long.parseLong(m.group(1));
result = new CalendarInterval(0, micros);
break;
}
default:
break;
}
} catch (Exception e) {
throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e);
}
}
return result;
}
/**
* Parse second_nano string in ss.nnnnnnnnn format to microseconds
*/
public static long parseSecondNano(String secondNano) throws IllegalArgumentException {
String[] parts = secondNano.split("\\.");
if (parts.length == 1) {
return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND,
Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND;
} else if (parts.length == 2) {
long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0],
Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND);
long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L);
return seconds * MICROS_PER_SECOND + nanos / 1000L;
} else {
throw new IllegalArgumentException(
"Interval string does not match second-nano format of ss.nnnnnnnnn");
}
}
public final int months;
public final long microseconds;
public CalendarInterval(int months, long microseconds) {
this.months = months;
this.microseconds = microseconds;
}
public CalendarInterval add(CalendarInterval that) {
int newMonths = this.months + that.months;
long newMicroseconds = this.microseconds + that.microseconds;
return new CalendarInterval(newMonths, newMicroseconds);
}
public CalendarInterval subtract(CalendarInterval that) {
int newMonths = this.months - that.months;
long newMicroseconds = this.microseconds - that.microseconds;
return new CalendarInterval(newMonths, newMicroseconds);
}
public CalendarInterval negate() {
return new CalendarInterval(-this.months, -this.microseconds);
}
@Override
public boolean equals(Object other) {
if (this == other) return true;
if (other == null || !(other instanceof CalendarInterval)) return false;
CalendarInterval o = (CalendarInterval) other;
return this.months == o.months && this.microseconds == o.microseconds;
}
@Override
public int hashCode() {
return 31 * months + (int) microseconds;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("interval");
if (months != 0) {
appendUnit(sb, months / 12, "year");
appendUnit(sb, months % 12, "month");
}
if (microseconds != 0) {
long rest = microseconds;
appendUnit(sb, rest / MICROS_PER_WEEK, "week");
rest %= MICROS_PER_WEEK;
appendUnit(sb, rest / MICROS_PER_DAY, "day");
rest %= MICROS_PER_DAY;
appendUnit(sb, rest / MICROS_PER_HOUR, "hour");
rest %= MICROS_PER_HOUR;
appendUnit(sb, rest / MICROS_PER_MINUTE, "minute");
rest %= MICROS_PER_MINUTE;
appendUnit(sb, rest / MICROS_PER_SECOND, "second");
rest %= MICROS_PER_SECOND;
appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond");
rest %= MICROS_PER_MILLI;
appendUnit(sb, rest, "microsecond");
}
return sb.toString();
}
private void appendUnit(StringBuilder sb, long value, String unit) {
if (value != 0) {
sb.append(' ').append(value).append(' ').append(unit).append('s');
}
}
}

View File

@@ -1,140 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* Created by zagnix on 2016/6/2.
*/
public class ServerPropertyConf {
private ConcurrentMap settings = new ConcurrentHashMap<String, String>();
public ServerPropertyConf() {
}
/**
* Set a configuration variable.
*/
public ServerPropertyConf set(String key, String value) {
set(key, value, false);
return this;
}
public ServerPropertyConf set(String key, String value, boolean silent) {
if (key == null) {
throw new NullPointerException("null key");
}
if (value == null) {
throw new NullPointerException("null value for " + key);
}
settings.put(key, value);
return this;
}
public long getSizeAsBytes(String s, long i) {
String value = (String) settings.get(s);
if (value != null) {
return byteStringAsBytes(value);
}
return i;
}
public long getSizeAsBytes(String s, String defaultValue) {
String value = (String) settings.get(s);
if (value != null) {
return byteStringAsBytes(value);
}
return byteStringAsBytes(defaultValue);
}
public double getDouble(String s, double v) {
return v;
}
public boolean getBoolean(String s, boolean b) {
String value = (String) settings.get(s);
if (value != null) {
return value.equals("true");
}
return b;
}
public long getLong(String s, long l) {
return l;
}
public boolean contains(String s) {
return true;
}
public int getInt(String s, int i) {
return i;
}
/**
* Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for internal use.
* <p>
* If no suffix is provided, the passed number is assumed to be in bytes.
*/
public Long byteStringAsBytes(String str) {
return JavaUtils.byteStringAsBytes(str);
}
/**
* Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for internal use.
* <p>
* If no suffix is provided, the passed number is assumed to be in kibibytes.
*/
public Long byteStringAsKb(String str) {
return JavaUtils.byteStringAsKb(str);
}
/**
* Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for internal use.
* <p>
* If no suffix is provided, the passed number is assumed to be in mebibytes.
*/
public Long byteStringAsMb(String str) {
return JavaUtils.byteStringAsMb(str);
}
/**
* Convert a passed byte string (e.g. 50b, 100k, or 250m, 500g) to gibibytes for internal use.
* <p>
* If no suffix is provided, the passed number is assumed to be in gibibytes.
*/
public Long byteStringAsGb(String str) {
return JavaUtils.byteStringAsGb(str);
}
/**
* Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes.
*/
public int memoryStringToMb(String str) {
// Convert to bytes, rather than directly to MB, because when no units are specified the unit
// is assumed to be bytes
return (int) (JavaUtils.byteStringAsBytes(str) / 1024 / 1024);
}
public String getString(String s, String defaultValue) {
String value = (String) settings.get(s);
if (value != null) {
return value;
}
return defaultValue;
}
}

View File

@@ -1,29 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils.sort;
import java.util.Iterator;
import java.util.NoSuchElementException;
/**
* Created by zagnix 2016/6/6.
*/
public class AbstractScalaRowIterator<T> implements Iterator<T> {
@Override
public boolean hasNext() {
return false;
}
@Override
public T next() {
throw new NoSuchElementException();
}
@Override
public void remove() {
}
}

View File

@@ -1,27 +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 com.actiontech.dble.memory.unsafe.utils.sort;
/**
* Compares 8-byte key prefixes in prefix sort. Subclasses may implement type-specific
* comparisons, such as lexicographic comparison for strings.
*/
public abstract class PrefixComparator {
public abstract int compare(long prefix1, long prefix2);
}

View File

@@ -1,154 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.types.ByteArray;
import com.google.common.primitives.UnsignedLongs;
public class PrefixComparators {
private PrefixComparators() {
}
public static final PrefixComparator STRING = new UnsignedPrefixComparator();
public static final PrefixComparator STRING_DESC = new UnsignedPrefixComparatorDesc();
public static final PrefixComparator BINARY = new UnsignedPrefixComparator();
public static final PrefixComparator BINARY_DESC = new UnsignedPrefixComparatorDesc();
public static final PrefixComparator LONG = new SignedPrefixComparator();
public static final PrefixComparator LONG_DESC = new SignedPrefixComparatorDesc();
public static final PrefixComparator DOUBLE = new UnsignedPrefixComparator();
public static final PrefixComparator DOUBLE_DESC = new UnsignedPrefixComparatorDesc();
public static final PrefixComparator RADIX_SORT_DEMO = new RadixSortDemo();
public static final class BinaryPrefixComparator {
public static long computePrefix(byte[] bytes) {
return ByteArray.getPrefix(bytes);
}
}
public static final class DoublePrefixComparator {
/**
* Converts the double into a value that compares correctly as an unsigned long. For more
* details see http://stereopsis.com/radix.html.
*/
public static long computePrefix(double value) {
// Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible
// positive NaN, so there's nothing special we need to do for NaNs.
long bits = Double.doubleToLongBits(value);
// Negative floats compare backwards due to their sign-magnitude representation, so flip
// all the bits in this case.
long mask = -(bits >>> 63) | 0x8000000000000000L;
return bits ^ mask;
}
}
/**
* Provides radix sort parameters. Comparators implementing this also are indicating that the
* ordering they define is compatible with radix sort.
*/
public abstract static class RadixSortSupport extends PrefixComparator {
/**
* @return Whether the sort should be descending in binary sort order.
*/
public abstract boolean sortDescending();
/**
* @return Whether the sort should take into account the sign bit.
*/
public abstract boolean sortSigned();
}
public static final class RadixSortDemo extends PrefixComparators.RadixSortSupport {
@Override
public boolean sortDescending() {
return false;
}
@Override
public boolean sortSigned() {
return false;
}
@Override
public int compare(long prefix1, long prefix2) {
return PrefixComparators.BINARY.compare(prefix1 & 0xffffff0000L, prefix1 & 0xffffff0000L);
}
}
//
// Standard prefix comparator implementations
//
public static final class UnsignedPrefixComparator extends RadixSortSupport {
@Override
public boolean sortDescending() {
return false;
}
@Override
public boolean sortSigned() {
return false;
}
@Override
public int compare(long aPrefix, long bPrefix) {
return UnsignedLongs.compare(aPrefix, bPrefix);
}
}
public static final class UnsignedPrefixComparatorDesc extends RadixSortSupport {
@Override
public boolean sortDescending() {
return true;
}
@Override
public boolean sortSigned() {
return false;
}
@Override
public int compare(long bPrefix, long aPrefix) {
return UnsignedLongs.compare(aPrefix, bPrefix);
}
}
public static final class SignedPrefixComparator extends RadixSortSupport {
@Override
public boolean sortDescending() {
return false;
}
@Override
public boolean sortSigned() {
return true;
}
@Override
public int compare(long a, long b) {
return Long.compare(a, b);
}
}
public static final class SignedPrefixComparatorDesc extends RadixSortSupport {
@Override
public boolean sortDescending() {
return true;
}
@Override
public boolean sortSigned() {
return true;
}
@Override
public int compare(long b, long a) {
return Long.compare(a, b);
}
}
}

View File

@@ -1,252 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.LongArray;
public final class RadixSort {
private RadixSort() {
}
/**
* Sorts a given array of longs using least-significant-digit radix sort. This routine assumes
* you have extra space at the end of the array at least equal to the number of records. The
* sort is destructive and may relocate the data positioned within the array.
*
* @param array array of long elements followed by at least that many empty slots.
* @param numRecords number of data records in the array.
* @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the
* least significant byte.
* @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the
* least significant byte. Must be greater than startByteIndex.
* @param desc whether this is a descending (binary-order) sort.
* @param signed whether this is a signed (two's complement) sort.
* @return The starting index of the sorted data within the given array. We return this instead
* of always copying the data back to position zero for efficiency.
*/
public static int sort(
LongArray array, int numRecords, int startByteIndex, int endByteIndex,
boolean desc, boolean signed) {
assert startByteIndex >= 0 : "startByteIndex (" + startByteIndex + ") should >= 0";
assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7";
assert endByteIndex > startByteIndex;
assert numRecords * 2L <= array.size();
int inIndex = 0;
int outIndex = numRecords;
if (numRecords > 0) {
long[][] counts = getCounts(array, numRecords, startByteIndex, endByteIndex);
for (int i = startByteIndex; i <= endByteIndex; i++) {
if (counts[i] != null) {
sortAtByte(
array, numRecords, counts[i], i, inIndex, outIndex,
desc, signed && i == endByteIndex);
int tmp = inIndex;
inIndex = outIndex;
outIndex = tmp;
}
}
}
return inIndex;
}
/**
* Performs a partial sort by copying data into destination offsets for each byte value at the
* specified byte offset.
*
* @param array array to partially sort.
* @param numRecords number of data records in the array.
* @param counts counts for each byte value. This routine destructively modifies this array.
* @param byteIdx the byte in a long to sort at, counting from the least significant byte.
* @param inIndex the starting index in the array where input data is located.
* @param outIndex the starting index where sorted output data should be written.
* @param desc whether this is a descending (binary-order) sort.
* @param signed whether this is a signed (two's complement) sort (only applies to last byte).
*/
private static void sortAtByte(
LongArray array, int numRecords, long[] counts, int byteIdx, int inIndex, int outIndex,
boolean desc, boolean signed) {
assert counts.length == 256;
long[] offsets = transformCountsToOffsets(
counts, numRecords, array.getBaseOffset() + outIndex * 8L, 8, desc, signed);
Object baseObject = array.getBaseObject();
long baseOffset = array.getBaseOffset() + inIndex * 8L;
long maxOffset = baseOffset + numRecords * 8L;
for (long offset = baseOffset; offset < maxOffset; offset += 8) {
long value = Platform.getLong(baseObject, offset);
int bucket = (int) ((value >>> (byteIdx * 8)) & 0xff);
Platform.putLong(baseObject, offsets[bucket], value);
offsets[bucket] += 8;
}
}
/**
* Computes a value histogram for each byte in the given array.
*
* @param array array to count records in.
* @param numRecords number of data records in the array.
* @param startByteIndex the first byte to compute counts for (the prior are skipped).
* @param endByteIndex the last byte to compute counts for.
* @return an array of eight 256-byte count arrays, one for each byte starting from the least
* significant byte. If the byte does not need sorting the array will be null.
*/
private static long[][] getCounts(LongArray array, int numRecords, int startByteIndex, int endByteIndex) {
long[][] counts = new long[8][];
// Optimization: do a fast pre-pass to determine which byte indices we can skip for sorting.
// If all the byte values at a particular index are the same we don't need to count it.
long bitwiseMax = 0;
long bitwiseMin = -1L;
long maxOffset = array.getBaseOffset() + 8L * numRecords;
Object baseObject = array.getBaseObject();
for (long offset = array.getBaseOffset(); offset < maxOffset; offset += 8) {
long value = Platform.getLong(baseObject, offset);
bitwiseMax |= value;
bitwiseMin &= value;
}
long bitsChanged = bitwiseMin ^ bitwiseMax;
// Compute counts for each byte index.
for (int i = startByteIndex; i <= endByteIndex; i++) {
if (((bitsChanged >>> (i * 8)) & 0xff) != 0) {
counts[i] = new long[256];
// TODO(ekl) consider computing all the counts in one pass.
for (long offset = array.getBaseOffset(); offset < maxOffset; offset += 8) {
counts[i][(int) ((Platform.getLong(baseObject, offset) >>> (i * 8)) & 0xff)]++;
}
}
}
return counts;
}
/**
* Transforms counts into the proper unsafe output offsets for the sort type.
*
* @param counts counts for each byte value. This routine destructively modifies this array.
* @param numRecords number of data records in the original data array.
* @param outputOffset output offset in bytes from the base array object.
* @param bytesPerRecord size of each record (8 for plain sort, 16 for key-prefix sort).
* @param desc whether this is a descending (binary-order) sort.
* @param signed whether this is a signed (two's complement) sort.
* @return the input counts array.
*/
private static long[] transformCountsToOffsets(
long[] counts, int numRecords, long outputOffset, int bytesPerRecord,
boolean desc, boolean signed) {
assert counts.length == 256;
int start = signed ? 128 : 0; // output the negative records first (values 129-255).
if (desc) {
long pos = numRecords;
for (int i = start; i < start + 256; i++) {
pos -= counts[i & 0xff];
counts[i & 0xff] = outputOffset + pos * bytesPerRecord;
}
} else {
long pos = 0;
for (int i = start; i < start + 256; i++) {
long tmp = counts[i & 0xff];
counts[i & 0xff] = outputOffset + pos * bytesPerRecord;
pos += tmp;
}
}
return counts;
}
/**
* Specialization of sort() for key-prefix arrays. In this type of array, each record consists
* of two longs, only the second of which is sorted on.
*/
public static int sortKeyPrefixArray(
LongArray array,
int numRecords,
int startByteIndex,
int endByteIndex,
boolean desc,
boolean signed) {
assert startByteIndex >= 0 : "startByteIndex (" + startByteIndex + ") should >= 0";
assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7";
assert endByteIndex > startByteIndex;
assert numRecords * 4L <= array.size();
int inIndex = 0;
int outIndex = numRecords * 2;
if (numRecords > 0) {
long[][] counts = getKeyPrefixArrayCounts(array, numRecords, startByteIndex, endByteIndex);
for (int i = startByteIndex; i <= endByteIndex; i++) {
if (counts[i] != null) {
sortKeyPrefixArrayAtByte(
array, numRecords, counts[i], i, inIndex, outIndex,
desc, signed && i == endByteIndex);
int tmp = inIndex;
inIndex = outIndex;
outIndex = tmp;
}
}
}
return inIndex;
}
/**
* Specialization of getCounts() for key-prefix arrays. We could probably combine this with
* getCounts with some added parameters but that seems to hurt in benchmarks.
*/
private static long[][] getKeyPrefixArrayCounts(
LongArray array, int numRecords, int startByteIndex, int endByteIndex) {
long[][] counts = new long[8][];
long bitwiseMax = 0;
long bitwiseMin = -1L;
long limit = array.getBaseOffset() + numRecords * 16L;
Object baseObject = array.getBaseObject();
for (long offset = array.getBaseOffset(); offset < limit; offset += 16) {
long value = Platform.getLong(baseObject, offset + 8);
bitwiseMax |= value;
bitwiseMin &= value;
}
long bitsChanged = bitwiseMin ^ bitwiseMax;
for (int i = startByteIndex; i <= endByteIndex; i++) {
if (((bitsChanged >>> (i * 8)) & 0xff) != 0) {
counts[i] = new long[256];
for (long offset = array.getBaseOffset(); offset < limit; offset += 16) {
counts[i][(int) ((Platform.getLong(baseObject, offset + 8) >>> (i * 8)) & 0xff)]++;
}
}
}
return counts;
}
/**
* Specialization of sortAtByte() for key-prefix arrays.
*/
private static void sortKeyPrefixArrayAtByte(
LongArray array, int numRecords, long[] counts, int byteIdx, int inIndex, int outIndex,
boolean desc, boolean signed) {
assert counts.length == 256;
long[] offsets = transformCountsToOffsets(
counts, numRecords, array.getBaseOffset() + outIndex * 8L, 16, desc, signed);
Object baseObject = array.getBaseObject();
long baseOffset = array.getBaseOffset() + inIndex * 8L;
long maxOffset = baseOffset + numRecords * 16L;
for (long offset = baseOffset; offset < maxOffset; offset += 16) {
long key = Platform.getLong(baseObject, offset);
long prefix = Platform.getLong(baseObject, offset + 8);
int bucket = (int) ((prefix >>> (byteIdx * 8)) & 0xff);
long dst = offsets[bucket];
Platform.putLong(baseObject, dst, key);
Platform.putLong(baseObject, dst + 8, prefix);
offsets[bucket] += 16;
}
}
}

View File

@@ -1,37 +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 com.actiontech.dble.memory.unsafe.utils.sort;
/**
* Compares records for ordering. In cases where the entire sorting key can fit in the 8-byte
* prefix, this may simply return 0.
*/
public abstract class RecordComparator {
/**
* Compare two records for order.
*
* @return a negative integer, zero, or a positive integer as the first record is less than,
* equal to, or greater than the second.
*/
public abstract int compare(
Object leftBaseObject,
long leftBaseOffset,
Object rightBaseObject,
long rightBaseOffset);
}

View File

@@ -1,46 +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 com.actiontech.dble.memory.unsafe.utils.sort;
public final class RecordPointerAndKeyPrefix {
private long recordPointer;
private long keyPrefix;
/**
* A pointer to a record;
*/
public long getRecordPointer() {
return recordPointer;
}
public void setRecordPointer(long recordPointer) {
this.recordPointer = recordPointer;
}
/**
* A key prefix, for use in comparisons.
*/
public long getKeyPrefix() {
return keyPrefix;
}
public void setKeyPrefix(long keyPrefix) {
this.keyPrefix = keyPrefix;
}
}

View File

@@ -1,100 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.row.StructType;
import com.actiontech.dble.memory.unsafe.row.UnsafeRow;
import com.actiontech.dble.memory.unsafe.utils.BytesTools;
import com.actiontech.dble.sqlengine.mpp.ColMeta;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import java.io.UnsupportedEncodingException;
/**
* Created by zagnix on 2016/6/20.
*/
public class RowPrefixComputer extends UnsafeExternalRowSorter.PrefixComputer {
private final ColMeta colMeta;
public RowPrefixComputer(StructType schema) {
/**
* get the index of the first key word of order
*/
int orderIndex = 0;
OrderCol[] orderCols = schema.getOrderCols();
if (orderCols != null) {
for (int i = 0; i < orderCols.length; i++) {
ColMeta meta = orderCols[i].colMeta;
if (meta.getColIndex() == 0) {
orderIndex = i;
break;
}
}
this.colMeta = orderCols[orderIndex].colMeta;
} else {
this.colMeta = null;
}
}
protected long computePrefix(UnsafeRow row) throws UnsupportedEncodingException {
if (this.colMeta == null) {
return 0;
}
int orderIndexType = colMeta.getColType();
byte[] rowIndexElem = null;
if (!row.isNullAt(colMeta.getColIndex())) {
rowIndexElem = row.getBinary(colMeta.getColIndex());
/**
* the first order by column
*/
switch (orderIndexType) {
case ColMeta.COL_TYPE_INT:
case ColMeta.COL_TYPE_LONG:
case ColMeta.COL_TYPE_INT24:
return BytesTools.getInt(rowIndexElem);
case ColMeta.COL_TYPE_SHORT:
return BytesTools.getShort(rowIndexElem);
case ColMeta.COL_TYPE_LONGLONG:
return BytesTools.getLong(rowIndexElem);
case ColMeta.COL_TYPE_FLOAT:
return PrefixComparators.DoublePrefixComparator.computePrefix(BytesTools.getFloat(rowIndexElem));
case ColMeta.COL_TYPE_DOUBLE:
case ColMeta.COL_TYPE_DECIMAL:
case ColMeta.COL_TYPE_NEWDECIMAL:
return PrefixComparators.DoublePrefixComparator.computePrefix(BytesTools.getDouble(rowIndexElem));
case ColMeta.COL_TYPE_DATE:
case ColMeta.COL_TYPE_TIMSTAMP:
case ColMeta.COL_TYPE_TIME:
case ColMeta.COL_TYPE_YEAR:
case ColMeta.COL_TYPE_DATETIME:
case ColMeta.COL_TYPE_NEWDATE:
case ColMeta.COL_TYPE_BIT:
case ColMeta.COL_TYPE_VAR_STRING:
case ColMeta.COL_TYPE_STRING:
// ENUM and SET ar all string
case ColMeta.COL_TYPE_ENUM:
case ColMeta.COL_TYPE_SET:
return PrefixComparators.BinaryPrefixComparator.computePrefix(rowIndexElem);
// not support BLOB,GEOMETRY
default:
break;
}
return 0;
} else {
rowIndexElem = new byte[1];
rowIndexElem[0] = UnsafeRow.NULL_MARK;
return PrefixComparators.BinaryPrefixComparator.computePrefix(rowIndexElem);
}
}
}

View File

@@ -1,79 +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 com.actiontech.dble.memory.unsafe.utils.sort;
/**
* Abstraction for sorting an arbitrary input buffer of data. This interface requires determining
* the sort key for a given element index, as well as swapping elements and moving data from one
* buffer to another.
* <p>
* Example format: an array of numbers, where each element is also the key.
* See [[KVArraySortDataFormat]] for a more exciting format.
* <p>
* Note: Declaring and instantiating multiple subclasses of this class would prevent JIT inlining
* overridden methods and hence decrease the shuffle performance.
*
* @tparam K Type of the sort key of each element
* @tparam Buffer Internal data structure used by a particular format (e.g., Array[Int]).
*/
// TODO: Making Buffer a real trait would be a better abstraction, but adds some complexity.
public abstract class SortDataFormat<K, B> {
/**
* Creates a new mutable key for reuse. This should be implemented if you want to override
* [[getKey(Buffer, Int, K)]].
*/
public abstract K newKey();
/**
* Return the sort key for the element at the given index.
*/
protected abstract K getKey(B data, int pos);
/**
* Returns the sort key for the element at the given index and reuse the input key if possible.
* The default implementation ignores the reuse parameter and invokes [[getKey(Buffer, Int]].
* If you want to override this method, you must implement [[newKey()]].
*/
protected K getKey(B data, int pos, K reuse) {
return getKey(data, pos);
}
/**
* Swap two elements.
*/
protected abstract void swap(B data, int pos0, int pos1);
/**
* Copy a single element from src(srcPos) to dst(dstPos).
*/
protected abstract void copyElement(B src, int srcPos, B dst, int dstPos);
/**
* Copy a range of elements starting at src(srcPos) to dst, starting at dstPos.
* Overlapping ranges are allowed.
*/
protected abstract void copyRange(B src, int srcPos, B dst, int dstPos, int length);
/**
* Allocates a Buffer that can hold up to 'length' elements.
* All elements of the buffer should be considered invalid until data is explicitly copied in.
*/
protected abstract B allocate(int length);
}

View File

@@ -1,41 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import java.util.Comparator;
/**
* A simple wrapper over the Java implementation [[TimSort]].
* <p>
* The Java implementation is package private, and hence it cannot be called outside package
*/
public class Sorter<K, B> {
private TimSort timSort = null;
public Sorter(SortDataFormat<K, B> s) {
timSort = new TimSort(s);
}
/**
* Sorts the input buffer within range [lo, hi).
*/
public void sort(B a, int lo, int hi, Comparator<K> c) {
timSort.sort(a, lo, hi, c);
}
}

View File

@@ -1,965 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
/*
* 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.
*/
/*
* Based on TimSort.java from the Android Open Source Project
*
* Copyright (C) 2008 The Android Open Source Project
*
* Licensed 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 com.actiontech.dble.memory.unsafe.utils.sort;
import java.util.Comparator;
/**
* A port of the Android TimSort class, which utilizes a "stable, adaptive, iterative mergesort."
* See the method comment on sort() for more details.
* <p>
* This has been kept in Java with the original style in order to match very closely with the
* Android source code, and thus be easy to verify correctness. The class is package private. We put
* a simple Scala wrapper {@link Sorter}, which is available to
* The purpose of the port is to generalize the interface to the sort to accept input data formats
* besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap
* uses this to sort an Array with alternating elements of the form [key, value, key, value].
* This generalization comes with minimal overhead -- see SortDataFormat for more information.
* <p>
* We allow key reuse to prevent creating many key objects -- see SortDataFormat.
*
* @see SortDataFormat
* @see Sorter
*/
class TimSort<K, B> {
/**
* This is the minimum sized sequence that will be merged. Shorter
* sequences will be lengthened by calling binarySort. If the entire
* array is less than this length, no merges will be performed.
* <p>
* This constant should be a power of two. It was 64 in Tim Peter's C
* implementation, but 32 was empirically determined to work better in
* this implementation. In the unlikely event that you set this constant
* to be a number that's not a power of two, you'll need to change the
* minRunLength computation.
* <p>
* If you decrease this constant, you must change the stackLen
* computation in the TimSort constructor, or you risk an
* ArrayOutOfBounds exception. See listsort.txt for a discussion
* of the minimum stack length required as a function of the length
* of the array being sorted and the minimum merge sequence length.
*/
private static final int MIN_MERGE = 32;
private final SortDataFormat<K, B> s;
TimSort(SortDataFormat<K, B> sortDataFormat) {
this.s = sortDataFormat;
}
/**
* A stable, adaptive, iterative mergesort that requires far fewer than
* n lg(n) comparisons when running on partially sorted arrays, while
* offering performance comparable to a traditional mergesort when run
* on random arrays. Like all proper mergesorts, this sort is stable and
* runs O(n log n) time (worst case). In the worst case, this sort requires
* temporary storage space for n/2 object references; in the best case,
* it requires only a small constant amount of space.
* <p>
* This implementation was adapted from Tim Peters's list sort for
* Python, which is described in detail here:
* <p>
* http://svn.python.org/projects/python/trunk/Objects/listsort.txt
* <p>
* Tim's C code may be found here:
* <p>
* http://svn.python.org/projects/python/trunk/Objects/listobject.c
* <p>
* The underlying techniques are described in this paper (and may have
* even earlier origins):
* <p>
* "Optimistic Sorting and Information Theoretic Complexity"
* Peter McIlroy
* SODA (Fourth Annual ACM-SIAM Symposium on Discrete Algorithms),
* pp 467-474, Austin, Texas, 25-27 January 1993.
* <p>
* While the API to this class consists solely of static methods, it is
* (privately) instantiable; a TimSort instance holds the state of an ongoing
* sort, assuming the input array is large enough to warrant the full-blown
* TimSort. Small arrays are sorted in place, using a binary insertion sort.
*
* @author Josh Bloch
*/
public void sort(B a, int lo, int hi, Comparator<? super K> c) {
assert c != null;
int nRemaining = hi - lo;
if (nRemaining < 2)
return; // Arrays of size 0 and 1 are always sorted
// If array is small, do a "mini-TimSort" with no merges
if (nRemaining < MIN_MERGE) {
int initRunLen = countRunAndMakeAscending(a, lo, hi, c);
binarySort(a, lo, hi, lo + initRunLen, c);
return;
}
/**
* March over the array once, left to right, finding natural runs,
* extending short natural runs to minRun elements, and merging runs
* to maintain stack invariant.
*/
SortState sortState = new SortState(a, c, hi - lo);
int minRun = minRunLength(nRemaining);
do {
// Identify next run
int runLen = countRunAndMakeAscending(a, lo, hi, c);
// If run is short, extend to min(minRun, nRemaining)
if (runLen < minRun) {
int force = nRemaining <= minRun ? nRemaining : minRun;
binarySort(a, lo, lo + force, lo + runLen, c);
runLen = force;
}
// Push run onto pending-run stack, and maybe merge
sortState.pushRun(lo, runLen);
sortState.mergeCollapse();
// Advance to find next run
lo += runLen;
nRemaining -= runLen;
} while (nRemaining != 0);
// Merge all remaining runs to complete sort
assert lo == hi;
sortState.mergeForceCollapse();
assert sortState.stackSize == 1;
}
/**
* Sorts the specified portion of the specified array using a binary
* insertion sort. This is the best method for sorting small numbers
* of elements. It requires O(n log n) compares, but O(n^2) data
* movement (worst case).
* <p>
* If the initial part of the specified range is already sorted,
* this method can take advantage of it: the method assumes that the
* elements from index {@code lo}, inclusive, to {@code start},
* exclusive are already sorted.
*
* @param a the array in which a range is to be sorted
* @param lo the index of the first element in the range to be sorted
* @param hi the index after the last element in the range to be sorted
* @param start the index of the first element in the range that is
* not already known to be sorted ({@code lo <= start <= hi})
* @param c comparator to used for the sort
*/
@SuppressWarnings("fallthrough")
private void binarySort(B a, int lo, int hi, int start, Comparator<? super K> c) {
assert lo <= start && start <= hi;
if (start == lo)
start++;
K key0 = s.newKey();
K key1 = s.newKey();
B pivotStore = s.allocate(1);
for (; start < hi; start++) {
s.copyElement(a, start, pivotStore, 0);
K pivot = s.getKey(pivotStore, 0, key0);
// Set left (and right) to the index where a[start] (pivot) belongs
int left = lo;
int right = start;
assert left <= right;
/*
* Invariants:
* pivot >= all in [lo, left).
* pivot < all in [right, start).
*/
while (left < right) {
int mid = (left + right) >>> 1;
if (c.compare(pivot, s.getKey(a, mid, key1)) < 0)
right = mid;
else
left = mid + 1;
}
assert left == right;
/*
* The invariants still hold: pivot >= all in [lo, left) and
* pivot < all in [left, start), so pivot belongs at left. Note
* that if there are elements equal to pivot, left points to the
* first slot after them -- that's why this sort is stable.
* Slide elements over to make room for pivot.
*/
int n = start - left; // The number of elements to move
// Switch is just an optimization for arraycopy in default case
switch (n) {
case 2:
s.copyElement(a, left + 1, a, left + 2);
s.copyElement(a, left, a, left + 1);
break;
case 1:
s.copyElement(a, left, a, left + 1);
break;
default:
s.copyRange(a, left, a, left + 1, n);
}
s.copyElement(pivotStore, 0, a, left);
}
}
/**
* Returns the length of the run beginning at the specified position in
* the specified array and reverses the run if it is descending (ensuring
* that the run will always be ascending when the method returns).
* <p>
* A run is the longest ascending sequence with:
* <p>
* a[lo] <= a[lo + 1] <= a[lo + 2] <= ...
* <p>
* or the longest descending sequence with:
* <p>
* a[lo] > a[lo + 1] > a[lo + 2] > ...
* <p>
* For its intended use in a stable mergesort, the strictness of the
* definition of "descending" is needed so that the call can safely
* reverse a descending sequence without violating stability.
*
* @param a the array in which a run is to be counted and possibly reversed
* @param lo index of the first element in the run
* @param hi index after the last element that may be contained in the run.
* It is required that {@code lo < hi}.
* @param c the comparator to used for the sort
* @return the length of the run beginning at the specified position in
* the specified array
*/
private int countRunAndMakeAscending(B a, int lo, int hi, Comparator<? super K> c) {
assert lo < hi;
int runHi = lo + 1;
if (runHi == hi)
return 1;
K key0 = s.newKey();
K key1 = s.newKey();
// Find end of run, and reverse range if descending
if (c.compare(s.getKey(a, runHi++, key0), s.getKey(a, lo, key1)) < 0) { // Descending
while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) < 0)
runHi++;
reverseRange(a, lo, runHi);
} else { // Ascending
while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0)
runHi++;
}
return runHi - lo;
}
/**
* Reverse the specified range of the specified array.
*
* @param a the array in which a range is to be reversed
* @param lo the index of the first element in the range to be reversed
* @param hi the index after the last element in the range to be reversed
*/
private void reverseRange(B a, int lo, int hi) {
hi--;
while (lo < hi) {
s.swap(a, lo, hi);
lo++;
hi--;
}
}
/**
* Returns the minimum acceptable run length for an array of the specified
* length. Natural runs shorter than this will be extended with
* {@link #binarySort}.
* <p>
* Roughly speaking, the computation is:
* <p>
* If n < MIN_MERGE, return n (it's too small to bother with fancy stuff).
* Else if n is an exact power of 2, return MIN_MERGE/2.
* Else return an int k, MIN_MERGE/2 <= k <= MIN_MERGE, such that n/k
* is close to, but strictly less than, an exact power of 2.
* <p>
* For the rationale, see listsort.txt.
*
* @param n the length of the array to be sorted
* @return the length of the minimum run to be merged
*/
private int minRunLength(int n) {
assert n >= 0;
int r = 0; // Becomes 1 if any 1 bits are shifted off
while (n >= MIN_MERGE) {
r |= (n & 1);
n >>= 1;
}
return n + r;
}
private final class SortState {
/**
* The Buffer being sorted.
*/
private final B a;
/**
* Length of the sort Buffer.
*/
private final int aLength;
/**
* The comparator for this sort.
*/
private final Comparator<? super K> c;
/**
* When we get into galloping mode, we stay there until both runs win less
* often than MIN_GALLOP consecutive times.
*/
private static final int MIN_GALLOP = 7;
/**
* This controls when we get *into* galloping mode. It is initialized
* to MIN_GALLOP. The mergeLo and mergeHi methods nudge it higher for
* random data, and lower for highly structured data.
*/
private int minGallop = MIN_GALLOP;
/**
* Maximum initial size of tmp array, which is used for merging. The array
* can grow to accommodate demand.
* <p>
* Unlike Tim's original C version, we do not allocate this much storage
* when sorting smaller arrays. This change was required for performance.
*/
private static final int INITIAL_TMP_STORAGE_LENGTH = 256;
/**
* Temp storage for merges.
*/
private B tmp; // Actual runtime type will be Object[], regardless of T
/**
* Length of the temp storage.
*/
private int tmpLength = 0;
/**
* A stack of pending runs yet to be merged. Run i starts at
* address base[i] and extends for len[i] elements. It's always
* true (so long as the indices are in bounds) that:
* <p>
* runBase[i] + runLen[i] == runBase[i + 1]
* <p>
* so we could cut the storage for this, but it's a minor amount,
* and keeping all the info explicit simplifies the code.
*/
private int stackSize = 0; // Number of pending runs on stack
private final int[] runBase;
private final int[] runLen;
/**
* Creates a TimSort instance to maintain the state of an ongoing sort.
*
* @param a the array to be sorted
* @param c the comparator to determine the order of the sort
*/
private SortState(B a, Comparator<? super K> c, int len) {
this.aLength = len;
this.a = a;
this.c = c;
// Allocate temp storage (which may be increased later if necessary)
tmpLength = len < 2 * INITIAL_TMP_STORAGE_LENGTH ? len >>> 1 : INITIAL_TMP_STORAGE_LENGTH;
tmp = s.allocate(tmpLength);
/*
* Allocate runs-to-be-merged stack (which cannot be expanded). The
* stack length requirements are described in listsort.txt. The C
* version always uses the same stack length (85), but this was
* measured to be too expensive when sorting "mid-sized" arrays (e.g.,
* 100 elements) in Java. Therefore, we use smaller (but sufficiently
* large) stack lengths for smaller arrays. The "magic numbers" in the
* computation below must be changed if MIN_MERGE is decreased. See
* the MIN_MERGE declaration above for more information.
*/
int stackLen = (len < 120 ? 5 :
len < 1542 ? 10 :
len < 119151 ? 19 : 40);
runBase = new int[stackLen];
runLen = new int[stackLen];
}
/**
* Pushes the specified run onto the pending-run stack.
*
* @param base index of the first element in the run
* @param len the number of elements in the run
*/
private void pushRun(int base, int len) {
this.runBase[stackSize] = base;
this.runLen[stackSize] = len;
stackSize++;
}
/**
* Examines the stack of runs waiting to be merged and merges adjacent runs
* until the stack invariants are reestablished:
* <p>
* 1. runLen[i - 3] > runLen[i - 2] + runLen[i - 1]
* 2. runLen[i - 2] > runLen[i - 1]
* <p>
* This method is called each time a new run is pushed onto the stack,
* so the invariants are guaranteed to hold for i < stackSize upon
* entry to the method.
*/
private void mergeCollapse() {
while (stackSize > 1) {
int n = stackSize - 2;
if ((n >= 1 && runLen[n - 1] <= runLen[n] + runLen[n + 1]) ||
(n >= 2 && runLen[n - 2] <= runLen[n] + runLen[n - 1])) {
if (runLen[n - 1] < runLen[n + 1])
n--;
} else if (runLen[n] > runLen[n + 1]) {
break; // Invariant is established
}
mergeAt(n);
}
}
/**
* Merges all runs on the stack until only one remains. This method is
* called once, to complete the sort.
*/
private void mergeForceCollapse() {
while (stackSize > 1) {
int n = stackSize - 2;
if (n > 0 && runLen[n - 1] < runLen[n + 1])
n--;
mergeAt(n);
}
}
/**
* Merges the two runs at stack indices i and i+1. Run i must be
* the penultimate or antepenultimate run on the stack. In other words,
* i must be equal to stackSize-2 or stackSize-3.
*
* @param i stack index of the first of the two runs to merge
*/
private void mergeAt(int i) {
assert stackSize >= 2;
assert i >= 0;
assert i == stackSize - 2 || i == stackSize - 3;
int base1 = runBase[i];
int len1 = runLen[i];
int base2 = runBase[i + 1];
int len2 = runLen[i + 1];
assert len1 > 0 && len2 > 0;
assert base1 + len1 == base2;
/*
* Record the length of the combined runs; if i is the 3rd-last
* run now, also slide over the last run (which isn't involved
* in this merge). The current run (i+1) goes away in any case.
*/
runLen[i] = len1 + len2;
if (i == stackSize - 3) {
runBase[i + 1] = runBase[i + 2];
runLen[i + 1] = runLen[i + 2];
}
stackSize--;
K key0 = s.newKey();
/*
* Find where the first element of run2 goes in run1. Prior elements
* in run1 can be ignored (because they're already in place).
*/
int k = gallopRight(s.getKey(a, base2, key0), a, base1, len1, 0, c);
assert k >= 0;
base1 += k;
len1 -= k;
if (len1 == 0)
return;
/*
* Find where the last element of run1 goes in run2. Subsequent elements
* in run2 can be ignored (because they're already in place).
*/
len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c);
assert len2 >= 0;
if (len2 == 0)
return;
// Merge remaining runs, using tmp array with min(len1, len2) elements
if (len1 <= len2)
mergeLo(base1, len1, base2, len2);
else
mergeHi(base1, len1, base2, len2);
}
/**
* Locates the position at which to insert the specified key into the
* specified sorted range; if the range contains an element equal to key,
* returns the index of the leftmost equal element.
*
* @param key the key whose insertion point to search for
* @param search the array in which to search
* @param base the index of the first element in the range
* @param len the length of the range; must be > 0
* @param hint the index at which to begin the search, 0 <= hint < n.
* The closer hint is to the result, the faster this method will run.
* @param comparator the comparator used to order the range, and to search
* @return the int k, 0 <= k <= n such that a[b + k - 1] < key <= a[b + k],
* pretending that a[b - 1] is minus infinity and a[b + n] is infinity.
* In other words, key belongs at index b + k; or in other words,
* the first k elements of a should precede key, and the last n - k
* should follow it.
*/
private int gallopLeft(K key, B search, int base, int len, int hint, Comparator<? super K> comparator) {
assert len > 0 && hint >= 0 && hint < len;
int lastOfs = 0;
int ofs = 1;
K key0 = s.newKey();
if (comparator.compare(key, s.getKey(search, base + hint, key0)) > 0) {
// Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs]
int maxOfs = len - hint;
while (ofs < maxOfs && comparator.compare(key, s.getKey(search, base + hint + ofs, key0)) > 0) {
lastOfs = ofs;
ofs = (ofs << 1) + 1;
if (ofs <= 0) // int overflow
ofs = maxOfs;
}
if (ofs > maxOfs)
ofs = maxOfs;
// Make offsets relative to base
lastOfs += hint;
ofs += hint;
} else { // key <= a[base + hint]
// Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs]
final int maxOfs = hint + 1;
while (ofs < maxOfs && comparator.compare(key, s.getKey(search, base + hint - ofs, key0)) <= 0) {
lastOfs = ofs;
ofs = (ofs << 1) + 1;
if (ofs <= 0) // int overflow
ofs = maxOfs;
}
if (ofs > maxOfs)
ofs = maxOfs;
// Make offsets relative to base
int tmpValue = lastOfs;
lastOfs = hint - ofs;
ofs = hint - tmpValue;
}
assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
/*
* Now a[base+lastOfs] < key <= a[base+ofs], so key belongs somewhere
* to the right of lastOfs but no farther right than ofs. Do a binary
* search, with invariant a[base + lastOfs - 1] < key <= a[base + ofs].
*/
lastOfs++;
while (lastOfs < ofs) {
int m = lastOfs + ((ofs - lastOfs) >>> 1);
if (comparator.compare(key, s.getKey(search, base + m, key0)) > 0)
lastOfs = m + 1; // a[base + m] < key
else
ofs = m; // key <= a[base + m]
}
assert lastOfs == ofs; // so a[base + ofs - 1] < key <= a[base + ofs]
return ofs;
}
/**
* Like gallopLeft, except that if the range contains an element equal to
* key, gallopRight returns the index after the rightmost equal element.
*
* @param key the key whose insertion point to search for
* @param search the array in which to search
* @param base the index of the first element in the range
* @param len the length of the range; must be > 0
* @param hint the index at which to begin the search, 0 <= hint < n.
* The closer hint is to the result, the faster this method will run.
* @param comparator the comparator used to order the range, and to search
* @return the int k, 0 <= k <= n such that a[b + k - 1] <= key < a[b + k]
*/
private int gallopRight(K key, B search, int base, int len, int hint, Comparator<? super K> comparator) {
assert len > 0 && hint >= 0 && hint < len;
int ofs = 1;
int lastOfs = 0;
K key1 = s.newKey();
if (comparator.compare(key, s.getKey(search, base + hint, key1)) < 0) {
// Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs]
int maxOfs = hint + 1;
while (ofs < maxOfs && comparator.compare(key, s.getKey(search, base + hint - ofs, key1)) < 0) {
lastOfs = ofs;
ofs = (ofs << 1) + 1;
if (ofs <= 0) // int overflow
ofs = maxOfs;
}
if (ofs > maxOfs)
ofs = maxOfs;
// Make offsets relative to b
int tmpValue = lastOfs;
lastOfs = hint - ofs;
ofs = hint - tmpValue;
} else { // a[b + hint] <= key
// Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs]
int maxOfs = len - hint;
while (ofs < maxOfs && comparator.compare(key, s.getKey(search, base + hint + ofs, key1)) >= 0) {
lastOfs = ofs;
ofs = (ofs << 1) + 1;
if (ofs <= 0) // int overflow
ofs = maxOfs;
}
if (ofs > maxOfs)
ofs = maxOfs;
// Make offsets relative to b
lastOfs += hint;
ofs += hint;
}
assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
/*
* Now a[b + lastOfs] <= key < a[b + ofs], so key belongs somewhere to
* the right of lastOfs but no farther right than ofs. Do a binary
* search, with invariant a[b + lastOfs - 1] <= key < a[b + ofs].
*/
lastOfs++;
while (lastOfs < ofs) {
int m = lastOfs + ((ofs - lastOfs) >>> 1);
if (comparator.compare(key, s.getKey(search, base + m, key1)) < 0)
ofs = m; // key < a[b + m]
else
lastOfs = m + 1; // a[b + m] <= key
}
assert lastOfs == ofs; // so a[b + ofs - 1] <= key < a[b + ofs]
return ofs;
}
/**
* Merges two adjacent runs in place, in a stable fashion. The first
* element of the first run must be greater than the first element of the
* second run (a[base1] > a[base2]), and the last element of the first run
* (a[base1 + len1-1]) must be greater than all elements of the second run.
* <p>
* For performance, this method should be called only when len1 <= len2;
* its twin, mergeHi should be called if len1 >= len2. (Either method
* may be called if len1 == len2.)
*
* @param base1 index of first element in first run to be merged
* @param len1 length of first run to be merged (must be > 0)
* @param base2 index of first element in second run to be merged
* (must be aBase + aLen)
* @param len2 length of second run to be merged (must be > 0)
*/
private void mergeLo(int base1, int len1, int base2, int len2) {
assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
// Copy first run into temp array
B src = this.a; // For performance
B dst = ensureCapacity(len1);
s.copyRange(src, base1, dst, 0, len1);
int cursor1 = 0; // Indexes into tmp array
int cursor2 = base2; // Indexes int a
int dst2 = base1; // Indexes int a
// Move first element of second run and deal with degenerate cases
s.copyElement(src, cursor2++, src, dst2++);
if (--len2 == 0) {
s.copyRange(dst, cursor1, src, dst2, len1);
return;
}
if (len1 == 1) {
s.copyRange(src, cursor2, src, dst2, len2);
s.copyElement(dst, cursor1, src, dst2 + len2); // Last elt of run 1 to end of merge
return;
}
K key0 = s.newKey();
K key1 = s.newKey();
Comparator<? super K> comparator = this.c; // Use local variable for performance
int gallop = this.minGallop; // " " " " "
outer:
while (true) {
int count1 = 0; // Number of times in a row that first run won
int count2 = 0; // Number of times in a row that second run won
/*
* Do the straightforward thing until (if ever) one run starts
* winning consistently.
*/
do {
assert len1 > 1 && len2 > 0;
if (comparator.compare(s.getKey(src, cursor2, key0), s.getKey(dst, cursor1, key1)) < 0) {
s.copyElement(src, cursor2++, src, dst2++);
count2++;
count1 = 0;
if (--len2 == 0)
break outer;
} else {
s.copyElement(dst, cursor1++, src, dst2++);
count1++;
count2 = 0;
if (--len1 == 1)
break outer;
}
} while ((count1 | count2) < gallop);
/*
* One run is winning so consistently that galloping may be a
* huge win. So try that, and continue galloping until (if ever)
* neither run appears to be winning consistently anymore.
*/
do {
assert len1 > 1 && len2 > 0;
count1 = gallopRight(s.getKey(src, cursor2, key0), dst, cursor1, len1, 0, comparator);
if (count1 != 0) {
s.copyRange(dst, cursor1, src, dst2, count1);
dst2 += count1;
cursor1 += count1;
len1 -= count1;
if (len1 <= 1) // len1 == 1 || len1 == 0
break outer;
}
s.copyElement(src, cursor2++, src, dst2++);
if (--len2 == 0)
break outer;
count2 = gallopLeft(s.getKey(dst, cursor1, key0), src, cursor2, len2, 0, comparator);
if (count2 != 0) {
s.copyRange(src, cursor2, src, dst2, count2);
dst2 += count2;
cursor2 += count2;
len2 -= count2;
if (len2 == 0)
break outer;
}
s.copyElement(dst, cursor1++, src, dst2++);
if (--len1 == 1)
break outer;
gallop--;
} while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
if (gallop < 0)
gallop = 0;
gallop += 2; // Penalize for leaving gallop mode
} // End of "outer" loop
this.minGallop = gallop < 1 ? 1 : gallop; // Write back to field
if (len1 == 1) {
assert len2 > 0;
s.copyRange(src, cursor2, src, dst2, len2);
s.copyElement(dst, cursor1, src, dst2 + len2); // Last elt of run 1 to end of merge
} else if (len1 == 0) {
throw new IllegalArgumentException(
"Comparison method violates its general contract!");
} else {
assert len2 == 0;
assert len1 > 1;
s.copyRange(dst, cursor1, src, dst2, len1);
}
}
/**
* Like mergeLo, except that this method should be called only if
* len1 >= len2; mergeLo should be called if len1 <= len2. (Either method
* may be called if len1 == len2.)
*
* @param base1 index of first element in first run to be merged
* @param len1 length of first run to be merged (must be > 0)
* @param base2 index of first element in second run to be merged
* (must be aBase + aLen)
* @param len2 length of second run to be merged (must be > 0)
*/
private void mergeHi(int base1, int len1, int base2, int len2) {
assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
// Copy second run into temp array
B src = this.a; // For performance
B dst = ensureCapacity(len2);
s.copyRange(src, base2, dst, 0, len2);
int cursor1 = base1 + len1 - 1; // Indexes into a
int cursor2 = len2 - 1; // Indexes into tmp array
int dst2 = base2 + len2 - 1; // Indexes into a
K key0 = s.newKey();
K key1 = s.newKey();
// Move last element of first run and deal with degenerate cases
s.copyElement(src, cursor1--, src, dst2--);
if (--len1 == 0) {
s.copyRange(dst, 0, src, dst2 - (len2 - 1), len2);
return;
}
if (len2 == 1) {
dst2 -= len1;
cursor1 -= len1;
s.copyRange(src, cursor1 + 1, src, dst2 + 1, len1);
s.copyElement(dst, cursor2, src, dst2);
return;
}
Comparator<? super K> comparator = this.c; // Use local variable for performance
int gallop = this.minGallop; // " " " " "
outer:
while (true) {
int count1 = 0; // Number of times in a row that first run won
int count2 = 0; // Number of times in a row that second run won
/*
* Do the straightforward thing until (if ever) one run
* appears to win consistently.
*/
do {
assert len1 > 0 && len2 > 1;
if (comparator.compare(s.getKey(dst, cursor2, key0), s.getKey(src, cursor1, key1)) < 0) {
s.copyElement(src, cursor1--, src, dst2--);
count1++;
count2 = 0;
if (--len1 == 0)
break outer;
} else {
s.copyElement(dst, cursor2--, src, dst2--);
count2++;
count1 = 0;
if (--len2 == 1)
break outer;
}
} while ((count1 | count2) < gallop);
/*
* One run is winning so consistently that galloping may be a
* huge win. So try that, and continue galloping until (if ever)
* neither run appears to be winning consistently anymore.
*/
do {
assert len1 > 0 && len2 > 1;
count1 = len1 - gallopRight(s.getKey(dst, cursor2, key0), src, base1, len1, len1 - 1, comparator);
if (count1 != 0) {
dst2 -= count1;
cursor1 -= count1;
len1 -= count1;
s.copyRange(src, cursor1 + 1, src, dst2 + 1, count1);
if (len1 == 0)
break outer;
}
s.copyElement(dst, cursor2--, src, dst2--);
if (--len2 == 1)
break outer;
count2 = len2 - gallopLeft(s.getKey(src, cursor1, key0), dst, 0, len2, len2 - 1, comparator);
if (count2 != 0) {
dst2 -= count2;
cursor2 -= count2;
len2 -= count2;
s.copyRange(dst, cursor2 + 1, src, dst2 + 1, count2);
if (len2 <= 1) // len2 == 1 || len2 == 0
break outer;
}
s.copyElement(src, cursor1--, src, dst2--);
if (--len1 == 0)
break outer;
gallop--;
} while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
if (gallop < 0)
gallop = 0;
gallop += 2; // Penalize for leaving gallop mode
} // End of "outer" loop
this.minGallop = gallop < 1 ? 1 : gallop; // Write back to field
if (len2 == 1) {
assert len1 > 0;
dst2 -= len1;
cursor1 -= len1;
s.copyRange(src, cursor1 + 1, src, dst2 + 1, len1);
s.copyElement(dst, cursor2, src, dst2); // Move first elt of run2 to front of merge
} else if (len2 == 0) {
throw new IllegalArgumentException(
"Comparison method violates its general contract!");
} else {
assert len1 == 0;
assert len2 > 0;
s.copyRange(dst, 0, src, dst2 - (len2 - 1), len2);
}
}
/**
* Ensures that the external array tmp has at least the specified
* number of elements, increasing its size if necessary. The size
* increases exponentially to ensure amortized linear time complexity.
*
* @param minCapacity the minimum required capacity of the tmp array
* @return tmp, whether or not it grew
*/
private B ensureCapacity(int minCapacity) {
if (tmpLength < minCapacity) {
// Compute smallest power of 2 > minCapacity
int newSize = minCapacity;
newSize |= newSize >> 1;
newSize |= newSize >> 2;
newSize |= newSize >> 4;
newSize |= newSize >> 8;
newSize |= newSize >> 16;
newSize++;
if (newSize < 0) // Not bloody likely!
newSize = minCapacity;
else
newSize = Math.min(newSize, aLength >>> 1);
tmp = s.allocate(newSize);
tmpLength = newSize;
}
return tmp;
}
}
}

View File

@@ -1,304 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.SeverMemory;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.row.StructType;
import com.actiontech.dble.memory.unsafe.row.UnsafeRow;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import com.actiontech.dble.sqlengine.mpp.RowDataPacketSorter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.Iterator;
import java.util.List;
public final class UnsafeExternalRowSorter {
private final Logger logger = LoggerFactory.getLogger(UnsafeExternalRowSorter.class);
private long numRowsInserted = 0;
private final StructType schema;
private final PrefixComputer prefixComputer;
private final UnsafeExternalSorter sorter;
private final PrefixComparator prefixComparator;
private final RecordComparator recordComparator;
public UnsafeExternalRowSorter(DataNodeMemoryManager dataNodeMemoryManager,
@Nonnull SeverMemory memory,
StructType schema,
PrefixComparator prefixComparator,
PrefixComputer prefixComputer,
long pageSizeBytes,
boolean canUseRadixSort,
boolean enableSort) throws IOException {
this.schema = schema;
this.prefixComputer = prefixComputer;
this.prefixComparator = prefixComparator;
this.recordComparator = new RowComparator(schema);
sorter = UnsafeExternalSorter.create(
dataNodeMemoryManager,
memory.getBlockManager(),
memory.getSerializerManager(),
recordComparator,
prefixComparator,
memory.getConf().getSizeAsBytes("server.pointer.array.len", "16K"),
pageSizeBytes,
canUseRadixSort,
enableSort);
}
public void insertRow(UnsafeRow row) throws IOException {
final long prefix = prefixComputer.computePrefix(row);
sorter.insertRecord(
row.getBaseObject(),
row.getBaseOffset(),
row.getSizeInBytes(),
prefix);
numRowsInserted++;
}
/**
* Return total rows
*/
public long getNumRowsInserted() {
return numRowsInserted;
}
/**
* Return the peak memory used so far, in bytes.
*/
public long getPeakMemoryUsage() {
return sorter.getPeakMemoryUsedBytes();
}
/**
* @return the total amount of time spent sorting data (in-memory only).
*/
public long getSortTimeNanos() {
return sorter.getSortTimeNanos();
}
public void cleanupResources() {
logger.debug("row sorter clean up resources!!!");
sorter.cleanupResources();
}
public Iterator<UnsafeRow> sort() throws IOException {
try {
final UnsafeSorterIterator sortedIterator = sorter.getSortedIterator();
if (!sortedIterator.hasNext()) {
cleanupResources();
}
return new AbstractScalaRowIterator<UnsafeRow>() {
private final int numFields = schema.length();
private UnsafeRow row = new UnsafeRow(numFields);
@Override
public boolean hasNext() {
return sortedIterator.hasNext();
}
@Override
public UnsafeRow next() {
try {
sortedIterator.loadNext();
row.pointTo(sortedIterator.getBaseObject(), sortedIterator.getBaseOffset(), sortedIterator.getRecordLength());
if (!hasNext()) {
UnsafeRow copy = row.copy(); // so that we don't have dangling pointers to freed page
row = null; // so that we don't keep references to the base object
cleanupResources();
return copy;
} else {
return row;
}
} catch (IOException e) {
cleanupResources();
// Scala iterators don't declare any checked exceptions, so we need to use this hack
// to re-throw the exception:
Platform.throwException(e);
}
throw new RuntimeException("Exception should have been re-thrown in next()");
}
@Override
public void remove() {
}
};
} catch (IOException e) {
cleanupResources();
throw e;
}
}
public Iterator<UnsafeRow> sort(Iterator<UnsafeRow> inputIterator) throws IOException {
while (inputIterator.hasNext()) {
insertRow(inputIterator.next());
}
return sort();
}
public UnsafeSorterIterator getRowUnsafeSorterIterator() throws IOException {
return sorter.getSortedIterator();
}
public Iterator<UnsafeRow> mergerSort(List<UnsafeSorterIterator> list) throws IOException {
UnsafeRowsMerger unsafeRowsMerger = new UnsafeRowsMerger(recordComparator, prefixComparator, list.size());
for (UnsafeSorterIterator aList : list) {
unsafeRowsMerger.addSpillIfNotEmpty(aList);
}
try {
final UnsafeSorterIterator sortedIterator = unsafeRowsMerger.getSortedIterator();
if (!sortedIterator.hasNext()) {
cleanupResources();
}
return new AbstractScalaRowIterator<UnsafeRow>() {
private final int numFields = schema.length();
private UnsafeRow row = new UnsafeRow(numFields);
@Override
public boolean hasNext() {
return sortedIterator.hasNext();
}
@Override
public UnsafeRow next() {
try {
sortedIterator.loadNext();
row.pointTo(
sortedIterator.getBaseObject(),
sortedIterator.getBaseOffset(),
sortedIterator.getRecordLength());
if (!hasNext()) {
UnsafeRow copy = row.copy(); // so that we don't have dangling pointers to freed page
row = null; // so that we don't keep references to the base object
cleanupResources();
return copy;
} else {
return row;
}
} catch (IOException e) {
cleanupResources();
// Scala iterators don't declare any checked exceptions, so we need to use this hack
// to re-throw the exception:
Platform.throwException(e);
}
throw new RuntimeException("Exception should have been re-thrown in next()");
}
@Override
public void remove() {
}
};
} catch (IOException e) {
cleanupResources();
throw e;
}
}
private static final class RowComparator extends RecordComparator {
private final int numFields;
private final UnsafeRow row1;
private final UnsafeRow row2;
private final StructType schema;
RowComparator(StructType schema) {
assert schema.length() >= 0;
this.schema = schema;
this.numFields = schema.length();
this.row1 = new UnsafeRow(numFields);
this.row2 = new UnsafeRow(numFields);
}
@Override
public int compare(Object baseObj1, long baseOff1, Object baseObj2, long baseOff2) {
OrderCol[] orderCols = schema.getOrderCols();
if (orderCols == null) {
return 0;
}
row1.pointTo(baseObj1, baseOff1, -1);
row2.pointTo(baseObj2, baseOff2, -1);
int cmp = 0;
int type = OrderCol.COL_ORDER_TYPE_ASC;
for (OrderCol orderCol : orderCols) {
int colIndex = orderCol.colMeta.getColIndex();
/**compare*/
byte[] left = null;
byte[] right = null;
if (!row1.isNullAt(colIndex)) {
left = row1.getBinary(colIndex);
} else {
left = new byte[1];
left[0] = UnsafeRow.NULL_MARK;
}
if (!row2.isNullAt(colIndex)) {
right = row2.getBinary(colIndex);
} else {
right = new byte[1];
right[0] = UnsafeRow.NULL_MARK;
}
if (orderCol.orderType == type) {
cmp = RowDataPacketSorter.compareObject(left, right, orderCol);
} else {
cmp = RowDataPacketSorter.compareObject(right, left, orderCol);
}
if (cmp != 0)
return cmp;
}
return cmp;
}
}
public abstract static class PrefixComputer {
protected abstract long computePrefix(UnsafeRow row) throws UnsupportedEncodingException;
}
}

View File

@@ -1,744 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.DbleServer;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.LongArray;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryConsumer;
import com.actiontech.dble.memory.unsafe.storage.DataNodeDiskManager;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.LinkedList;
import java.util.Queue;
/**
* External sorter based on {@link UnsafeInMemorySorter}.
*/
public final class UnsafeExternalSorter extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
@Nullable
private final PrefixComparator prefixComparator;
@Nullable
private final RecordComparator recordComparator;
private final DataNodeMemoryManager dataNodeMemoryManager;
private final DataNodeDiskManager blockManager;
private final SerializerManager serializerManager;
/**
* The buffer size to use when writing spills using DiskRowWriter
*/
private final int fileBufferSizeBytes;
/**
* Memory pages that hold the records being sorted. The pages in this list are freed when
* spilling, although in principle we could recycle these pages across spills (on the other hand,
* this might not be necessary if we maintained a pool of re-usable pages in the DataNodeMemoryManager
* itself).
*/
private final LinkedList<MemoryBlock> allocatedPages = new LinkedList<>();
private final LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
// These variables are reset after spilling:
@Nullable
private volatile UnsafeInMemorySorter inMemSorter;
private MemoryBlock currentPage = null;
private long pageCursor = -1;
private long peakMemoryUsedBytes = 0;
private long totalSpillBytes = 0L;
private long totalSortTimeNanos = 0L;
private volatile SpillableIterator readingIterator = null;
public static UnsafeExternalSorter createWithExistingInMemorySorter(
DataNodeMemoryManager dataNodeMemoryManager,
DataNodeDiskManager blockManager,
SerializerManager serializerManager,
RecordComparator recordComparator,
PrefixComparator prefixComparator,
int initialSize,
long pageSizeBytes,
UnsafeInMemorySorter inMemorySorter, boolean enableSort) throws IOException {
UnsafeExternalSorter sorter = new UnsafeExternalSorter(dataNodeMemoryManager, blockManager,
serializerManager, recordComparator, prefixComparator, initialSize,
pageSizeBytes, inMemorySorter, false /* ignored */, enableSort);
sorter.spill(Long.MAX_VALUE, sorter);
// The external sorter will be used to insert records, in-memory sorter is not needed.
sorter.inMemSorter = null;
return sorter;
}
public static UnsafeExternalSorter create(
DataNodeMemoryManager dataNodeMemoryManager,
DataNodeDiskManager blockManager,
SerializerManager serializerManager,
RecordComparator recordComparator,
PrefixComparator prefixComparator,
long initialSize,
long pageSizeBytes,
boolean canUseRadixSort,
boolean enableSort) {
return new UnsafeExternalSorter(dataNodeMemoryManager, blockManager, serializerManager, recordComparator, prefixComparator, initialSize, pageSizeBytes, null,
canUseRadixSort, enableSort);
}
private UnsafeExternalSorter(
DataNodeMemoryManager dataNodeMemoryManager,
DataNodeDiskManager blockManager,
SerializerManager serializerManager,
RecordComparator recordComparator,
PrefixComparator prefixComparator,
long initialSize,
long pageSizeBytes,
@Nullable UnsafeInMemorySorter existingInMemorySorter,
boolean canUseRadixSort, boolean enableSort) {
super(dataNodeMemoryManager, pageSizeBytes);
this.dataNodeMemoryManager = dataNodeMemoryManager;
this.blockManager = blockManager;
this.serializerManager = serializerManager;
this.recordComparator = recordComparator;
this.prefixComparator = prefixComparator;
if (DbleServer.getInstance().getServerMemory() != null) {
this.fileBufferSizeBytes = (int) DbleServer.getInstance().
getServerMemory().getConf().getSizeAsBytes("server.merge.file.buffer", "32k");
} else {
this.fileBufferSizeBytes = 32 * 1024;
}
if (existingInMemorySorter == null) {
this.inMemSorter = new UnsafeInMemorySorter(
this, dataNodeMemoryManager, recordComparator, prefixComparator, initialSize, canUseRadixSort, enableSort);
} else {
this.inMemSorter = existingInMemorySorter;
}
this.peakMemoryUsedBytes = getMemoryUsage();
}
/**
* Marks the current page as no-more-space-available, and as a result, either allocate a
* new page or spill when we see the next record.
*/
@VisibleForTesting
public void closeCurrentPage() {
if (currentPage != null) {
pageCursor = currentPage.getBaseOffset() + currentPage.size();
}
}
/**
* Sort and spill the current records in response to memory pressure.
*/
@Override
public long spill(long size, MemoryConsumer trigger) throws IOException {
if (trigger != this) {
if (readingIterator != null) {
return readingIterator.spill();
}
return 0L; // this should throw exception
}
if (inMemSorter == null || inMemSorter.numRecords() <= 0) {
return 0L;
}
logger.info("Thread" + Thread.currentThread().getId() + " spilling sort data of " + JavaUtils.bytesToString(getMemoryUsage()) +
" to disk (" + spillWriters.size() + " times so far)");
// We only write out contents of the inMemSorter if it is not empty.
if (inMemSorter.numRecords() > 0) {
/**
* create an SpillWriter, the data has ordered in memory,write it to the disk.
*/
final UnsafeSorterSpillWriter spillWriter = new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, /**writeMetrics,*/inMemSorter.numRecords());
/**
* add to SpillWriter list,mark that there are spillWriters.size() have written to disk
*/
spillWriters.add(spillWriter);
/**
* In Memory Sort use tim sorter or radix sorter
*/
final UnsafeSorterIterator sortedRecords = inMemSorter.getSortedIterator();
/**
* write rows to disk one by one
*/
while (sortedRecords.hasNext()) {
/**
*
*/
sortedRecords.loadNext();
/**
* get the object of the recod which is in the beginning of records,on-heap is obj,off-heap is null
*/
final Object baseObject = sortedRecords.getBaseObject();
/**
* get the offset to the beginning record
*/
final long baseOffset = sortedRecords.getBaseOffset();
/**
* record the length
*/
final int recordLength = sortedRecords.getRecordLength();
/**
* Write a record to a spill file.
*/
spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix());
}
/**
* close the spillWriter
*/
spillWriter.close();
}
/**
* release the memory of sorter
*/
final long spillSize = freeMemory();
// Note that this is more-or-less going to be a multiple of the page size, so wasted space in
// pages will currently be counted as memory spilled even though that space isn't actually
// written to disk. This also counts the space needed to store the sorter's pointer array.
inMemSorter.reset();
// Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the
// records. Otherwise, if the task is over allocated memory, then without freeing the memory
// pages, we might not be able to get memory for the pointer array.
totalSpillBytes += spillSize;
return spillSize;
}
/**
* Return the total memory usage of this sorter, including the data pages and the sorter's pointer
* array.
*/
private long getMemoryUsage() {
long totalPageSize = 0;
for (MemoryBlock page : allocatedPages) {
totalPageSize += page.size();
}
return ((inMemSorter == null) ? 0 : inMemSorter.getMemoryUsage()) + totalPageSize;
}
private void updatePeakMemoryUsed() {
long mem = getMemoryUsage();
if (mem > peakMemoryUsedBytes) {
peakMemoryUsedBytes = mem;
}
}
/**
* Return the peak memory used so far, in bytes.
*/
public long getPeakMemoryUsedBytes() {
updatePeakMemoryUsed();
return peakMemoryUsedBytes;
}
/**
* @return the total amount of time spent sorting data (in-memory only).
*/
public long getSortTimeNanos() {
UnsafeInMemorySorter sorter = inMemSorter;
if (sorter != null) {
return sorter.getSortTimeNanos();
}
return totalSortTimeNanos;
}
/**
* Return the total number of bytes that has been spilled into disk so far.
*/
public long getSpillSize() {
return totalSpillBytes;
}
@VisibleForTesting
public int getNumberOfAllocatedPages() {
return allocatedPages.size();
}
/**
* Free this sorter's data pages.
*
* @return the number of bytes freed.
*/
private long freeMemory() {
updatePeakMemoryUsed();
long memoryFreed = 0;
for (MemoryBlock block : allocatedPages) {
memoryFreed += block.size();
freePage(block);
}
allocatedPages.clear();
currentPage = null;
pageCursor = 0;
return memoryFreed;
}
/**
* Deletes any spill files created by this sorter.
*/
private void deleteSpillFiles() {
for (UnsafeSorterSpillWriter spill : spillWriters) {
File file = spill.getFile();
if (file == null)
continue;
try {
JavaUtils.deleteRecursively(file.getParentFile().getParentFile());
} catch (IOException e) {
logger.error(e.getMessage());
}
if (file.exists()) {
if (!file.delete()) {
logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
}
}
}
}
/**
* Frees this sorter's in-memory data structures and cleans up its spill files.
*/
public void cleanupResources() {
synchronized (this) {
deleteSpillFiles();
freeMemory();
if (inMemSorter != null) {
inMemSorter.free();
inMemSorter = null;
}
}
}
/**
* Checks whether there is enough space to insert an additional record in to the sort pointer
* array and grows the array if additional space is required. If the required space cannot be
* obtained, then the in-memory data will be spilled to disk.
*/
private void growPointerArrayIfNecessary() throws IOException {
assert (inMemSorter != null);
if (!inMemSorter.hasSpaceForAnotherRecord()) {
long used = inMemSorter.getMemoryUsage();
LongArray array;
try {
// could trigger spilling
array = allocateLongArray(used / 8 * 2);
} catch (OutOfMemoryError e) {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
throw e;
}
return;
}
// check if spilling is triggered or not
if (inMemSorter.hasSpaceForAnotherRecord()) {
freeLongArray(array);
} else {
inMemSorter.expandPointerArray(array);
}
}
}
/**
* Allocates more memory in order to insert an additional record. This will request additional
* memory from the memory manager and spill if the requested memory can not be obtained.
*
* @param required the required space in the data page, in bytes, including space for storing
* the record size. This must be less than or equal to the page size (records
* that exceed the page size are handled via a different code path which uses
* special overflow pages).
*/
private void acquireNewPageIfNecessary(int required) {
if (currentPage == null ||
pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) {
// TODO: try to find space on previous pages
currentPage = allocatePage(required);
pageCursor = currentPage.getBaseOffset();
allocatedPages.add(currentPage);
}
}
/**
* Write a record to the sorter.
*/
public void insertRecord(Object recordBase, long recordOffset, int length, long prefix)
throws IOException {
growPointerArrayIfNecessary();
// Need 4 bytes to store the record length.
final int required = length + 4;
acquireNewPageIfNecessary(required);
final Object base = currentPage.getBaseObject();
final long recordAddress = dataNodeMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
Platform.putInt(base, pageCursor, length);
pageCursor += 4;
Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length);
pageCursor += length;
assert (inMemSorter != null);
inMemSorter.insertRecord(recordAddress, prefix);
}
/**
* Write a key-value record to the sorter. The key and value will be put together in-memory,
* using the following format:
* <p>
* record length (4 bytes), key length (4 bytes), key data, value data
* <p>
* record length = key length + value length + 4
*/
public void insertKVRecord(Object keyBase, long keyOffset, int keyLen,
Object valueBase, long valueOffset, int valueLen, long prefix)
throws IOException {
growPointerArrayIfNecessary();
final int required = keyLen + valueLen + 4 + 4;
acquireNewPageIfNecessary(required);
/**
* insert k-v into currentPage(MemoryBlock),the position is pageCursor
*/
final Object base = currentPage.getBaseObject();
/**
* record address according to currentPage and pageCursor
*/
final long recordAddress = dataNodeMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
/**
* total length of a row =keyLen + valueLen + record length (4 for int)
*/
Platform.putInt(base, pageCursor, keyLen + valueLen + 4/**record length*/);
/**
* add 4 bytes
*/
pageCursor += 4;
/**
* the size of key len
*/
Platform.putInt(base, pageCursor, keyLen);
/**
* add 4 bytes
*/
pageCursor += 4;
/**
* record the key
*/
Platform.copyMemory(keyBase, keyOffset, base, pageCursor, keyLen);
/**
* add keyLen bytes
*/
pageCursor += keyLen;
/**
* record the value
*/
Platform.copyMemory(valueBase, valueOffset, base, pageCursor, valueLen);
/**
* add valueLen bytes
*/
pageCursor += valueLen;
assert (inMemSorter != null);
/**
* insert the pointer into the longArray
* longArray point to the real value of the pointer in the Page
*/
inMemSorter.insertRecord(recordAddress, prefix);
}
/**
* Merges another UnsafeExternalSorters into this one, the other one will be emptied.
*
* @throws IOException
*/
public void merge(UnsafeExternalSorter other) throws IOException {
other.spill();
spillWriters.addAll(other.spillWriters);
// remove them from `spillWriters`, or the files will be deleted in `cleanupResources`.
other.spillWriters.clear();
other.cleanupResources();
}
/**
* SpillableIterator is an Iterator for order in memory /disk?
* Returns a sorted iterator. It is the caller's responsibility to call `cleanupResources()`
* after consuming this iterator.
*/
public UnsafeSorterIterator getSortedIterator() throws IOException {
assert (recordComparator != null);
if (spillWriters.isEmpty()) {
assert (inMemSorter != null);
readingIterator = new SpillableIterator(inMemSorter.getSortedIterator());
return readingIterator;
} else {
/**
* merger files of UnsafeSorterSpillWriter and order it
*/
final UnsafeSorterSpillMerger spillMerger =
new UnsafeSorterSpillMerger(recordComparator, prefixComparator, spillWriters.size());
for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
/**
* add to UnsafeSorterSpillMerger from UnsafeSorterSpillReader
*/
spillMerger.addSpillIfNotEmpty(spillWriter.getReader(serializerManager));
}
if (inMemSorter != null) {
readingIterator = new SpillableIterator(inMemSorter.getSortedIterator());
spillMerger.addSpillIfNotEmpty(readingIterator);
}
/**
* sort
*/
return spillMerger.getSortedIterator();
}
}
/**
* Returns a iterator, which will return the rows in the order as inserted.
* <p>
* It is the caller's responsibility to call `cleanupResources()`
* after consuming this iterator.
* <p>
* TODO: support forced spilling
*/
public UnsafeSorterIterator getIterator() throws IOException {
/**
* if spillWriters is empty,read from memory
*/
if (spillWriters.isEmpty()) {
assert (inMemSorter != null);
return inMemSorter.getSortedIterator();
} else {
/**
* read data from file of spillWriters to UnsafeSorterIterator
* and add to the queue
*/
LinkedList<UnsafeSorterIterator> queue = new LinkedList<>();
for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
queue.add(spillWriter.getReader(serializerManager));
}
if (inMemSorter != null) {
queue.add(inMemSorter.getSortedIterator());
}
/**
* merge multi-UnsafeSorterIterator to single UnsafeSorterIterator
*/
return new ChainedIterator(queue);
}
}
/**
* An UnsafeSorterIterator that support spilling.
*/
public class SpillableIterator extends UnsafeSorterIterator {
private UnsafeSorterIterator upstream;
private UnsafeSorterIterator nextUpstream = null;
private MemoryBlock lastPage = null;
private boolean loaded = false;
private int numRecords = 0;
SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) {
this.upstream = inMemIterator;
this.numRecords = inMemIterator.getNumRecords();
}
public int getNumRecords() {
return numRecords;
}
public long spill() throws IOException {
synchronized (this) {
if (!(upstream instanceof UnsafeInMemorySorter.SortedIterator && nextUpstream == null && numRecords > 0)) {
return 0L;
}
UnsafeInMemorySorter.SortedIterator inMemIterator =
((UnsafeInMemorySorter.SortedIterator) upstream).clone();
// Iterate over the records that have not been returned and spill them.
final UnsafeSorterSpillWriter spillWriter =
new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, /**writeMetrics,*/numRecords);
while (inMemIterator.hasNext()) {
inMemIterator.loadNext();
final Object baseObject = inMemIterator.getBaseObject();
final long baseOffset = inMemIterator.getBaseOffset();
final int recordLength = inMemIterator.getRecordLength();
spillWriter.write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix());
}
spillWriter.close();
spillWriters.add(spillWriter);
nextUpstream = spillWriter.getReader(serializerManager);
long released = 0L;
synchronized (UnsafeExternalSorter.this) {
// release the pages except the one that is used. There can still be a caller that
// is accessing the current record. We free this page in that caller's next loadNext()
// call.
for (MemoryBlock page : allocatedPages) {
if (!loaded || page.getBaseObject() != upstream.getBaseObject()) {
released += page.size();
freePage(page);
} else {
lastPage = page;
}
}
allocatedPages.clear();
}
// in-memory sorter will not be used after spilling
assert (inMemSorter != null);
released += inMemSorter.getMemoryUsage();
totalSortTimeNanos += inMemSorter.getSortTimeNanos();
inMemSorter.free();
inMemSorter = null;
totalSpillBytes += released;
return released;
}
}
@Override
public boolean hasNext() {
return numRecords > 0;
}
@Override
public void loadNext() throws IOException {
synchronized (this) {
loaded = true;
if (nextUpstream != null) {
// Just consumed the last record from in memory iterator
if (lastPage != null) {
freePage(lastPage);
lastPage = null;
}
upstream = nextUpstream;
nextUpstream = null;
}
numRecords--;
upstream.loadNext();
}
}
@Override
public Object getBaseObject() {
return upstream.getBaseObject();
}
@Override
public long getBaseOffset() {
return upstream.getBaseOffset();
}
@Override
public int getRecordLength() {
return upstream.getRecordLength();
}
@Override
public long getKeyPrefix() {
return upstream.getKeyPrefix();
}
}
/**
* Chain multiple UnsafeSorterIterator together as single one.
*/
static class ChainedIterator extends UnsafeSorterIterator {
private final Queue<UnsafeSorterIterator> iterators;
private UnsafeSorterIterator current;
private int numRecords;
ChainedIterator(Queue<UnsafeSorterIterator> iterators) {
assert iterators.size() > 0;
this.numRecords = 0;
for (UnsafeSorterIterator iterator : iterators) {
this.numRecords += iterator.getNumRecords();
}
this.iterators = iterators;
this.current = iterators.remove();
}
@Override
public int getNumRecords() {
return numRecords;
}
@Override
public boolean hasNext() {
while (!current.hasNext() && !iterators.isEmpty()) {
current = iterators.remove();
}
return current.hasNext();
}
@Override
public void loadNext() throws IOException {
while (!current.hasNext() && !iterators.isEmpty()) {
current = iterators.remove();
}
current.loadNext();
}
@Override
public Object getBaseObject() {
return current.getBaseObject();
}
@Override
public long getBaseOffset() {
return current.getBaseOffset();
}
@Override
public int getRecordLength() {
return current.getRecordLength();
}
@Override
public long getKeyPrefix() {
return current.getKeyPrefix();
}
}
}

View File

@@ -1,319 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.LongArray;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryConsumer;
import javax.annotation.Nullable;
import java.util.Comparator;
/**
* Sorts records using an AlphaSort-style key-prefix sort. This sort stores pointers to records
* alongside a user-defined prefix of the record's sorting key. When the underlying sort algorithm
* compares records, it will first compare the stored key prefixes; if the prefixes are not equal,
* then we do not need to traverse the record pointers to compare the actual records. Avoiding these
* random memory accesses improves cache hit rates.
*/
public final class UnsafeInMemorySorter {
private final MemoryConsumer consumer;
private final DataNodeMemoryManager memoryManager;
@Nullable
private final Sorter<RecordPointerAndKeyPrefix, LongArray> sorter;
@Nullable
private final Comparator<RecordPointerAndKeyPrefix> sortComparator;
/**
* If non-null, specifies the radix sort parameters and that radix sort will be used.
*/
@Nullable
private final PrefixComparators.RadixSortSupport radixSortSupport;
/**
* Set to 2x for radix sort to reserve extra memory for sorting, otherwise 1x.
*/
private final int memoryAllocationFactor;
/**
* Within this buffer, position {@code 2 * i} holds a pointer pointer to the record at
* index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix.
*/
private LongArray array;
/**
* The position in the sort buffer where new records can be inserted.
*/
private int pos = 0;
private long initialSize;
private long totalSortTimeNanos = 0L;
private boolean enableSort = true;
public UnsafeInMemorySorter(
final MemoryConsumer consumer,
final DataNodeMemoryManager memoryManager,
final RecordComparator recordComparator,
final PrefixComparator prefixComparator,
long initialSize,
boolean canUseRadixSort, boolean enableSort) {
this(consumer, memoryManager, recordComparator, prefixComparator,
consumer.allocateLongArray(initialSize * 2), canUseRadixSort, enableSort);
}
public UnsafeInMemorySorter(
final MemoryConsumer consumer,
final DataNodeMemoryManager memoryManager,
final RecordComparator recordComparator,
final PrefixComparator prefixComparator,
LongArray array,
boolean canUseRadixSort,
boolean enableSort) {
this.consumer = consumer;
this.memoryManager = memoryManager;
this.initialSize = array.size();
if (recordComparator != null) {
this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
if (canUseRadixSort && prefixComparator instanceof PrefixComparators.RadixSortSupport) {
this.radixSortSupport = (PrefixComparators.RadixSortSupport) prefixComparator;
} else {
this.radixSortSupport = null;
}
} else {
this.sorter = null;
this.sortComparator = null;
this.radixSortSupport = null;
}
this.enableSort = enableSort;
this.memoryAllocationFactor = this.radixSortSupport != null ? 2 : 1;
this.array = array;
}
/**
* Free the memory used by pointer array.
*/
public void free() {
if (consumer != null) {
consumer.freeLongArray(array);
array = null;
}
}
public void reset() {
if (consumer != null) {
consumer.freeLongArray(array);
this.array = consumer.allocateLongArray(initialSize);
}
pos = 0;
}
/**
* @return the number of records that have been inserted into this sorter.
*/
public int numRecords() {
return pos / 2;
}
/**
* @return the total amount of time spent sorting data (in-memory only).
*/
public long getSortTimeNanos() {
return totalSortTimeNanos;
}
public long getMemoryUsage() {
return array.size() * 8;
}
public boolean hasSpaceForAnotherRecord() {
return pos + 1 < (array.size() / memoryAllocationFactor);
}
public void expandPointerArray(LongArray newArray) {
if (newArray.size() < array.size()) {
throw new OutOfMemoryError("Not enough memory to grow pointer array");
}
Platform.copyMemory(
array.getBaseObject(),
array.getBaseOffset(),
newArray.getBaseObject(),
newArray.getBaseOffset(),
array.size() * (8 / memoryAllocationFactor));
consumer.freeLongArray(array);
array = newArray;
}
/**
* Inserts a record to be sorted. Assumes that the record pointer points to a record length
* stored as a 4-byte integer, followed by the record's bytes.
*
* @param recordPointer pointer to a record in a data page, encoded by {@link DataNodeMemoryManager}.
* @param keyPrefix a user-defined key prefix
*/
public void insertRecord(long recordPointer, long keyPrefix) {
if (!hasSpaceForAnotherRecord()) {
throw new IllegalStateException("There is no space for new record");
}
/**
* INSERT recordPointer FIRST,THEN keyPrefix
* */
array.set(pos, recordPointer);
pos++;
array.set(pos, keyPrefix);
pos++;
}
/**
* Return an iterator over record pointers in sorted order. For efficiency, all calls to
* {@code next()} will return the same mutable object.
*/
public SortedIterator getSortedIterator() {
int offset = 0;
long start = System.nanoTime();
if (sorter != null && enableSort) {
if (this.radixSortSupport != null) {
// TODO(ekl) we should handle NULL values before radix sort for efficiency, since they
// force a full-width sort (and we cannot radix-sort nullable long fields at all).
offset = RadixSort.sortKeyPrefixArray(array, pos / 2, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned());
} else {
sorter.sort(array, 0, pos / 2, sortComparator);
}
}
totalSortTimeNanos += System.nanoTime() - start;
return new SortedIterator(pos / 2, offset);
}
/**
* Return an iterator over record pointers int not sorted order. For efficiency, all calls to
* {@code next()} will return the same mutable object.
*/
public SortedIterator getIterator() {
return new SortedIterator(pos / 2, 0);
}
public final class SortedIterator extends UnsafeSorterIterator implements Cloneable {
private final int numRecords;
private int position;
private int offset;
private Object baseObject;
private long baseOffset;
private long keyPrefix;
private int recordLength;
private SortedIterator(int numRecords, int offset) {
this.numRecords = numRecords;
this.position = 0;
this.offset = offset;
}
public SortedIterator clone() {
SortedIterator iterator = new SortedIterator(numRecords, offset);
iterator.position = position;
iterator.baseObject = baseObject;
iterator.baseOffset = baseOffset;
iterator.keyPrefix = keyPrefix;
iterator.recordLength = recordLength;
return iterator;
}
@Override
public int getNumRecords() {
return numRecords;
}
@Override
public boolean hasNext() {
return position / 2 < numRecords;
}
@Override
public void loadNext() {
// This pointer points to a 4-byte record length, followed by the record's bytes
final long recordPointer = array.get(offset + position);
baseObject = memoryManager.getPage(recordPointer);
baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length
recordLength = Platform.getInt(baseObject, baseOffset - 4);
keyPrefix = array.get(offset + position + 1);
position += 2;
}
@Override
public Object getBaseObject() {
return baseObject;
}
@Override
public long getBaseOffset() {
return baseOffset;
}
@Override
public int getRecordLength() {
return recordLength;
}
@Override
public long getKeyPrefix() {
return keyPrefix;
}
}
private static final class SortComparator implements Comparator<RecordPointerAndKeyPrefix> {
private final RecordComparator recordComparator;
private final PrefixComparator prefixComparator;
private final DataNodeMemoryManager memoryManager;
SortComparator(
RecordComparator recordComparator,
PrefixComparator prefixComparator,
DataNodeMemoryManager memoryManager) {
this.recordComparator = recordComparator;
this.prefixComparator = prefixComparator;
this.memoryManager = memoryManager;
}
@Override
public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) {
final int prefixComparisonResult = prefixComparator.compare(r1.getKeyPrefix(), r2.getKeyPrefix());
if (prefixComparisonResult == 0) {
final Object baseObject1 = memoryManager.getPage(r1.getRecordPointer());
final long baseOffset1 = memoryManager.getOffsetInPage(r1.getRecordPointer()) + 4; // skip length
final Object baseObject2 = memoryManager.getPage(r2.getRecordPointer());
final long baseOffset2 = memoryManager.getOffsetInPage(r2.getRecordPointer()) + 4; // skip length
return recordComparator.compare(baseObject1, baseOffset1, baseObject2, baseOffset2);
} else {
return prefixComparisonResult;
}
}
}
}

View File

@@ -1,106 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.utils.sort;
import java.io.IOException;
import java.util.Comparator;
import java.util.PriorityQueue;
/**
* Created by zagnix on 2016/6/25.
*/
public final class UnsafeRowsMerger {
private int numRecords = 0;
private final PriorityQueue<UnsafeSorterIterator> priorityQueue;
UnsafeRowsMerger(
final RecordComparator recordComparator,
final PrefixComparator prefixComparator,
final int numSpills) {
final Comparator<UnsafeSorterIterator> comparator = new Comparator<UnsafeSorterIterator>() {
@Override
public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) {
final int prefixComparisonResult = prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
if (prefixComparisonResult == 0) {
return recordComparator.compare(
left.getBaseObject(), left.getBaseOffset(),
right.getBaseObject(), right.getBaseOffset());
} else {
return prefixComparisonResult;
}
}
};
/**
* use priorityQueue to order the Spill File
* and it can write to file if finished.
*/
priorityQueue = new PriorityQueue<>(numSpills, comparator);
}
/**
* Add an UnsafeSorterIterator to this merger
*/
public void addSpillIfNotEmpty(UnsafeSorterIterator iterator) throws IOException {
if (iterator.hasNext()) {
iterator.loadNext();
priorityQueue.add(iterator);
numRecords += iterator.getNumRecords();
}
}
public int getNumRecords() {
return numRecords;
}
public UnsafeSorterIterator getSortedIterator() throws IOException {
return new UnsafeSorterIterator() {
private UnsafeSorterIterator spillReader;
@Override
public int getNumRecords() {
return numRecords;
}
@Override
public boolean hasNext() {
return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext());
}
@Override
public void loadNext() throws IOException {
if (spillReader != null) {
if (spillReader.hasNext()) {
spillReader.loadNext();
priorityQueue.add(spillReader);
}
}
spillReader = priorityQueue.remove();
}
@Override
public Object getBaseObject() {
return spillReader.getBaseObject();
}
@Override
public long getBaseOffset() {
return spillReader.getBaseOffset();
}
@Override
public int getRecordLength() {
return spillReader.getRecordLength();
}
@Override
public long getKeyPrefix() {
return spillReader.getKeyPrefix();
}
};
}
}

View File

@@ -1,91 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.LongArray;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
/**
* Supports sorting an array of (record pointer, key prefix) pairs.
* Used in {@link UnsafeInMemorySorter}.
* <p>
* Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at
* index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix.
*/
public final class UnsafeSortDataFormat
extends SortDataFormat<RecordPointerAndKeyPrefix, LongArray> {
public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat();
private UnsafeSortDataFormat() {
}
@Override
public RecordPointerAndKeyPrefix getKey(LongArray data, int pos) {
// Since we re-use keys, this method shouldn't be called.
throw new UnsupportedOperationException();
}
@Override
public RecordPointerAndKeyPrefix getKey(LongArray data, int pos,
RecordPointerAndKeyPrefix reuse) {
reuse.setRecordPointer(data.get(pos * 2));
reuse.setKeyPrefix(data.get(pos * 2 + 1));
return reuse;
}
@Override
public RecordPointerAndKeyPrefix newKey() {
return new RecordPointerAndKeyPrefix();
}
@Override
public void swap(LongArray data, int pos0, int pos1) {
long tempPointer = data.get(pos0 * 2);
long tempKeyPrefix = data.get(pos0 * 2 + 1);
data.set(pos0 * 2, data.get(pos1 * 2));
data.set(pos0 * 2 + 1, data.get(pos1 * 2 + 1));
data.set(pos1 * 2, tempPointer);
data.set(pos1 * 2 + 1, tempKeyPrefix);
}
@Override
public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) {
dst.set(dstPos * 2, src.get(srcPos * 2));
dst.set(dstPos * 2 + 1, src.get(srcPos * 2 + 1));
}
@Override
public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) {
Platform.copyMemory(
src.getBaseObject(),
src.getBaseOffset() + srcPos * 16L,
dst.getBaseObject(),
dst.getBaseOffset() + dstPos * 16L,
length * 16L);
}
@Override
public LongArray allocate(int length) {
assert (length < Integer.MAX_VALUE / 2) : "Length " + length + " is too large";
return new LongArray(MemoryBlock.fromLongArray(new long[length * 2]));
}
}

View File

@@ -1,37 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import java.io.IOException;
public abstract class UnsafeSorterIterator {
public abstract boolean hasNext();
public abstract void loadNext() throws IOException;
public abstract Object getBaseObject();
public abstract long getBaseOffset();
public abstract int getRecordLength();
public abstract long getKeyPrefix();
public abstract int getNumRecords();
}

View File

@@ -1,129 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import java.io.IOException;
import java.util.Comparator;
import java.util.PriorityQueue;
final class UnsafeSorterSpillMerger {
private int numRecords = 0;
private final PriorityQueue<UnsafeSorterIterator> priorityQueue;
UnsafeSorterSpillMerger(
final RecordComparator recordComparator,
final PrefixComparator prefixComparator,
final int numSpills) {
final Comparator<UnsafeSorterIterator> comparator = new Comparator<UnsafeSorterIterator>() {
@Override
public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) {
final int prefixComparisonResult = prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
if (prefixComparisonResult == 0) {
return recordComparator.compare(
left.getBaseObject(), left.getBaseOffset(),
right.getBaseObject(), right.getBaseOffset());
} else {
return prefixComparisonResult;
}
}
};
priorityQueue = new PriorityQueue<>(numSpills, comparator);
}
/**
* Add an UnsafeSorterIterator to this merger
*/
public void addSpillIfNotEmpty(UnsafeSorterIterator spillReader) throws IOException {
if (spillReader.hasNext()) {
// We only add the spillReader to the priorityQueue if it is not empty. We do this to
// make sure the hasNext method of UnsafeSorterIterator returned by getSortedIterator
// does not return wrong result because hasNext will returns true
// at least priorityQueue.size() times. If we allow n spillReaders in the
// priorityQueue, we will have n extra empty records in the result of UnsafeSorterIterator.
spillReader.loadNext();
priorityQueue.add(spillReader);
numRecords += spillReader.getNumRecords();
}
}
/**
* @return
* @throws IOException
*/
public UnsafeSorterIterator getSortedIterator() throws IOException {
return new UnsafeSorterIterator() {
private UnsafeSorterIterator spillReader;
@Override
public int getNumRecords() {
return numRecords;
}
@Override
public boolean hasNext() {
return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext());
}
@Override
public void loadNext() throws IOException {
if (spillReader != null) {
if (spillReader.hasNext()) {
spillReader.loadNext();
/**
* priorityQueue will change the order in it.
* when spillReader add to it
* the smallest ready to pop forASC
*/
priorityQueue.add(spillReader);
}
}
/**
* pop from the queue, spillreader
*/
spillReader = priorityQueue.remove();
}
@Override
public Object getBaseObject() {
return spillReader.getBaseObject();
}
@Override
public long getBaseOffset() {
return spillReader.getBaseOffset();
}
@Override
public int getRecordLength() {
return spillReader.getRecordLength();
}
@Override
public long getKeyPrefix() {
return spillReader.getKeyPrefix();
}
};
}
}

View File

@@ -1,119 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.storage.ConnectionId;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import java.io.*;
/**
* Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description
* of the file format).
*/
public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable {
private InputStream in;
private DataInputStream din;
// Variables that change with every record read:
private int recordLength;
private long keyPrefix;
private int numRecords;
private int numRecordsRemaining;
private byte[] arr = new byte[1024 * 1024];
private Object baseObject = arr;
private final long baseOffset = Platform.BYTE_ARRAY_OFFSET;
public UnsafeSorterSpillReader(
SerializerManager serializerManager,
File file,
ConnectionId blockId) throws IOException {
assert (file.length() > 0);
final BufferedInputStream bs = new BufferedInputStream(new FileInputStream(file));
try {
this.in = serializerManager.wrapForCompression(blockId, bs);
this.din = new DataInputStream(this.in);
numRecords = numRecordsRemaining = din.readInt();
} catch (IOException e) {
Closeables.close(bs, /* swallowIOException = */ true);
throw e;
}
}
@Override
public int getNumRecords() {
return numRecords;
}
@Override
public boolean hasNext() {
return (numRecordsRemaining > 0);
}
@Override
public void loadNext() throws IOException {
recordLength = din.readInt();
keyPrefix = din.readLong();
if (recordLength > arr.length) {
arr = new byte[recordLength];
baseObject = arr;
}
ByteStreams.readFully(in, arr, 0, recordLength);
numRecordsRemaining--;
if (numRecordsRemaining == 0) {
close();
}
}
@Override
public Object getBaseObject() {
return baseObject;
}
@Override
public long getBaseOffset() {
return baseOffset;
}
@Override
public int getRecordLength() {
return recordLength;
}
@Override
public long getKeyPrefix() {
return keyPrefix;
}
@Override
public void close() throws IOException {
if (in != null) {
try {
in.close();
} finally {
in = null;
din = null;
}
}
}
}

View File

@@ -1,193 +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 com.actiontech.dble.memory.unsafe.utils.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.storage.*;
import java.io.File;
import java.io.IOException;
/**
* Spills a list of sorted records to disk. Spill files have the following format:
* <p>
* [# of records (int)] [[len (int)][prefix (long)][data (bytes)]...]
*/
public final class UnsafeSorterSpillWriter {
static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024;
// Small writes to DiskRowWriter will be fairly inefficient. Since there doesn't seem to
// be an API to directly transfer bytes from managed memory to the disk writer, we buffer
// data through a byte array.
private byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE];
private final File file;
private final ConnectionId conId;
private final int numRecordsToWrite;
private DiskRowWriter writer;
private int numRecordsSpilled = 0;
public UnsafeSorterSpillWriter(
DataNodeDiskManager blockManager,
int fileBufferSize,
int numRecordsToWrite) throws IOException {
DataNodeFileManager diskBlockManager = blockManager.diskBlockManager();
this.conId = diskBlockManager.createTempLocalBlock();
this.file = diskBlockManager.getFile(this.conId);
this.numRecordsToWrite = numRecordsToWrite;
// Unfortunately, we need a serializer instance in order to construct a DiskRowWriter.
// Our write path doesn't actually use this serializer (since we end up calling the `write()`
// OutputStream methods), but DiskRowWriter still calls some methods on it. To work
// around this, we pass a dummy no-op serializer.
writer = blockManager.getDiskWriter(file, DummySerializerInstance.INSTANCE, fileBufferSize/**,writeMetrics*/);
// Write the number of records
writeIntToBuffer(numRecordsToWrite, 0);
writer.write(writeBuffer, 0, 4);
}
// Based on DataOutputStream.writeLong.
private void writeLongToBuffer(long v, int offset) throws IOException {
writeBuffer[offset + 0] = (byte) (v >>> 56);
writeBuffer[offset + 1] = (byte) (v >>> 48);
writeBuffer[offset + 2] = (byte) (v >>> 40);
writeBuffer[offset + 3] = (byte) (v >>> 32);
writeBuffer[offset + 4] = (byte) (v >>> 24);
writeBuffer[offset + 5] = (byte) (v >>> 16);
writeBuffer[offset + 6] = (byte) (v >>> 8);
writeBuffer[offset + 7] = (byte) (v >>> 0);
}
// Based on DataOutputStream.writeInt.
private void writeIntToBuffer(int v, int offset) throws IOException {
writeBuffer[offset + 0] = (byte) (v >>> 24);
writeBuffer[offset + 1] = (byte) (v >>> 16);
writeBuffer[offset + 2] = (byte) (v >>> 8);
writeBuffer[offset + 3] = (byte) (v >>> 0);
}
/**
* Write a record to a spill file.
*
* @param baseObject the base object / memory page containing the record
* @param baseOffset the base offset which points directly to the record data.
* @param recordLength the length of the record.
* @param keyPrefix a sort key prefix
*/
public void write(
Object baseObject,
long baseOffset,
int recordLength,
long keyPrefix) throws IOException {
if (numRecordsSpilled == numRecordsToWrite) {
throw new IllegalStateException(
"Number of records written exceeded numRecordsToWrite = " + numRecordsToWrite);
} else {
numRecordsSpilled++;
}
/**
* [# of records (int)] [[len (int)][prefix (long)][data (bytes)]...]
* a row format
* */
/**
* recordLength 4bytes
*/
writeIntToBuffer(recordLength, 0);
/**
* key,8bytes
*/
writeLongToBuffer(keyPrefix, 4);
/**
* dataRemaining real data bytes
*/
int dataRemaining = recordLength;
/**
* freeSpace
*/
int freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE - 4 - 8; // space used by prefix + len
/**
*recordReadPosition
*/
long recordReadPosition = baseOffset;
while (dataRemaining > 0) {
/**
* read real data ,min(freeSpaceInWriteBuffer,dataRemaining)
*/
final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining);
/**
* copy from baseObjectto writeBuffer
*/
Platform.copyMemory(
baseObject, /**srd*/
recordReadPosition, /**offset*/
writeBuffer, /**write dst*/
Platform.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer), /**write offset*/
toTransfer);
/**
* write writeBufferto disk
*/
writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer) + toTransfer);
/**
* add toTransfer size
*/
recordReadPosition += toTransfer;
/**
* calc dataRemainingrecord
*/
dataRemaining -= toTransfer;
/**
* init the WriteBuffer to DISK_WRITE_BUFFER_SIZE
*/
freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE;
}
/**
* write the remain data to disk
*/
if (freeSpaceInWriteBuffer < DISK_WRITE_BUFFER_SIZE) {
writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer));
}
writer.recordWritten();
}
public void close() throws IOException {
writer.commitAndClose();
writer = null;
writeBuffer = null;
}
public File getFile() {
return file;
}
public UnsafeSorterSpillReader getReader(SerializerManager serializerManager) throws IOException {
return new UnsafeSorterSpillReader(serializerManager, file, conId);
}
}

View File

@@ -8,7 +8,6 @@ package com.actiontech.dble.net.mysql;
import com.actiontech.dble.backend.mysql.BufferUtil;
import com.actiontech.dble.config.Fields;
import com.actiontech.dble.memory.unsafe.row.UnsafeRow;
import com.actiontech.dble.net.FrontendConnection;
import com.actiontech.dble.util.ByteUtil;
import com.actiontech.dble.util.DateUtil;
@@ -50,34 +49,6 @@ public class BinaryRowDataPacket extends MySQLPacket {
public BinaryRowDataPacket() {
}
/**
* transform from UnsafeRow to BinaryRowDataPacket
* <p>
* Notice: when <b>isOffHeapuseOffHeapForMerge</b>is enable,
* UnsafeRow package the data,
* so now need to unpackage to BinaryRowDataPacket
*
* @param fields
* @param unsafeRow
*/
public void read(List<FieldPacket> fields, UnsafeRow unsafeRow) {
this.fieldPackets = fields;
this.fieldCount = unsafeRow.numFields();
this.fieldValues = new ArrayList<>(fieldCount);
this.nullBitMap = new byte[(fieldCount + 7 + 2) / 8];
for (int i = 0; i < this.fieldCount; i++) {
byte[] fv = unsafeRow.getBinary(i);
FieldPacket fieldPk = fields.get(i);
if (fv == null) {
storeNullBitMap(i);
this.fieldValues.add(fv);
} else {
convert(fv, fieldPk);
}
}
}
/**
* transfor from RowDataPacket to BinaryRowDataPacket
*

View File

@@ -20,7 +20,6 @@ import com.actiontech.dble.route.function.AbstractPartitionAlgorithm;
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
import com.actiontech.dble.route.util.RouterUtil;
import com.actiontech.dble.server.ServerConnection;
import com.actiontech.dble.server.parser.ServerParse;
import com.actiontech.dble.server.util.GlobalTableUtil;
import com.actiontech.dble.server.util.SchemaUtil;
import com.actiontech.dble.server.util.SchemaUtil.SchemaInfo;

View File

@@ -20,7 +20,6 @@ import com.actiontech.dble.route.parser.druid.ReplaceTemp;
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
import com.actiontech.dble.route.util.RouterUtil;
import com.actiontech.dble.server.ServerConnection;
import com.actiontech.dble.server.parser.ServerParse;
import com.actiontech.dble.server.util.GlobalTableUtil;
import com.actiontech.dble.server.util.SchemaUtil;
import com.actiontech.dble.server.util.SchemaUtil.SchemaInfo;

View File

@@ -1,69 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* based on code by MyCATCopyrightHolder Copyright (c) 2013, OpenCloudDB/MyCAT.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp;
import java.io.Serializable;
public class ColMeta implements Serializable {
public static final int COL_TYPE_DECIMAL = 0;
public static final int COL_TYPE_INT = 1;
public static final int COL_TYPE_SHORT = 2;
public static final int COL_TYPE_LONG = 3;
public static final int COL_TYPE_FLOAT = 4;
public static final int COL_TYPE_DOUBLE = 5;
public static final int COL_TYPE_NULL = 6;
public static final int COL_TYPE_TIMSTAMP = 7;
public static final int COL_TYPE_LONGLONG = 8;
public static final int COL_TYPE_INT24 = 9;
public static final int COL_TYPE_DATE = 0x0a;
public static final int COL_TYPE_DATETIME = 0X0C;
public static final int COL_TYPE_TIME = 0x0b;
public static final int COL_TYPE_YEAR = 0x0d;
public static final int COL_TYPE_NEWDATE = 0x0e;
public static final int COL_TYPE_VACHAR = 0x0f;
public static final int COL_TYPE_BIT = 0x10;
public static final int COL_TYPE_NEWDECIMAL = 0xf6;
public static final int COL_TYPE_ENUM = 0xf7;
public static final int COL_TYPE_SET = 0xf8;
public static final int COL_TYPE_TINY_BLOB = 0xf9;
public static final int COL_TYPE_TINY_TYPE_MEDIUM_BLOB = 0xfa;
public static final int COL_TYPE_TINY_TYPE_LONG_BLOB = 0xfb;
public static final int COL_TYPE_BLOB = 0xfc;
public static final int COL_TYPE_VAR_STRING = 0xfd;
public static final int COL_TYPE_STRING = 0xfe;
public static final int COL_TYPE_GEOMETRY = 0xff;
private int colIndex;
private final int colType;
private int decimals;
public ColMeta(int colIndex, int colType) {
super();
this.colIndex = colIndex;
this.colType = colType;
}
public int getColIndex() {
return colIndex;
}
public int getColType() {
return colType;
}
@Override
public String toString() {
return "ColMeta [colIndex=" + colIndex + ", colType=" + colType + "]";
}
public int getDecimals() {
return decimals;
}
public void setDecimals(int decimals) {
this.decimals = decimals;
}
}

View File

@@ -1,33 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* based on code by MyCATCopyrightHolder Copyright (c) 2013, OpenCloudDB/MyCAT.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp;
public class OrderCol {
public final int orderType;
public final ColMeta colMeta;
public static final int COL_ORDER_TYPE_ASC = 0; // ASC
public static final int COL_ORDER_TYPE_DESC = 1; // DESC
public OrderCol(ColMeta colMeta) {
this(colMeta, COL_ORDER_TYPE_ASC);
}
public OrderCol(ColMeta colMeta, int orderType) {
super();
this.colMeta = colMeta;
this.orderType = orderType;
}
public int getOrderType() {
return orderType;
}
public ColMeta getColMeta() {
return colMeta;
}
}

View File

@@ -1,35 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp;
/**
* Created by zagnix on 2016/7/6.
*/
/**
* PackWraper konws its datanode.
*/
public final class PackWraper {
private byte[] rowData;
private String dataNode;
public byte[] getRowData() {
return rowData;
}
public void setRowData(byte[] rowData) {
this.rowData = rowData;
}
public String getDataNode() {
return dataNode;
}
public void setDataNode(String dataNode) {
this.dataNode = dataNode;
}
}

View File

@@ -1,177 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* based on code by MyCATCopyrightHolder Copyright (c) 2013, OpenCloudDB/MyCAT.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp;
import com.actiontech.dble.memory.unsafe.utils.BytesTools;
import com.actiontech.dble.net.mysql.RowDataPacket;
import com.actiontech.dble.util.ByteUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.ConcurrentLinkedQueue;
public class RowDataPacketSorter {
private static final Logger LOGGER = LoggerFactory.getLogger(RowDataPacketSorter.class);
protected final OrderCol[] orderCols;
private Collection<RowDataPacket> sorted = new ConcurrentLinkedQueue<>();
private RowDataPacket[] array, resultTemp;
private int p1, pr, p2;
public RowDataPacketSorter(OrderCol[] orderCols) {
super();
this.orderCols = orderCols;
}
public boolean addRow(RowDataPacket row) {
return this.sorted.add(row);
}
public Collection<RowDataPacket> getSortedResult() {
try {
this.mergeSort(sorted.toArray(new RowDataPacket[sorted.size()]));
} catch (Exception e) {
LOGGER.error("getSortedResultError", e);
}
if (array != null) {
Collections.addAll(this.sorted, array);
}
return sorted;
}
private RowDataPacket[] mergeSort(RowDataPacket[] result) throws Exception {
this.sorted.clear();
array = result;
if (result == null || result.length < 2 || this.orderCols == null || orderCols.length < 1) {
return result;
}
mergeR(0, result.length - 1);
return array;
}
private void mergeR(int startIndex, int endIndex) {
if (startIndex < endIndex) {
int mid = (startIndex + endIndex) / 2;
mergeR(startIndex, mid);
mergeR(mid + 1, endIndex);
merge(startIndex, mid, endIndex);
}
}
private void merge(int startIndex, int midIndex, int endIndex) {
resultTemp = new RowDataPacket[(endIndex - startIndex + 1)];
pr = 0;
p1 = startIndex;
p2 = midIndex + 1;
while (p1 <= midIndex || p2 <= endIndex) {
if (p1 == midIndex + 1) {
while (p2 <= endIndex) {
resultTemp[pr++] = array[p2++];
}
} else if (p2 == endIndex + 1) {
while (p1 <= midIndex) {
resultTemp[pr++] = array[p1++];
}
} else {
compare(0);
}
}
for (p1 = startIndex, p2 = 0; p1 <= endIndex; p1++, p2++) {
array[p1] = resultTemp[p2];
}
}
/**
* @param byColumnIndex
*/
private void compare(int byColumnIndex) {
if (byColumnIndex == this.orderCols.length) {
if (this.orderCols[byColumnIndex - 1].orderType == OrderCol.COL_ORDER_TYPE_ASC) {
resultTemp[pr++] = array[p1++];
} else {
resultTemp[pr++] = array[p2++];
}
return;
}
byte[] left = array[p1].fieldValues.get(this.orderCols[byColumnIndex].colMeta.getColIndex());
byte[] right = array[p2].fieldValues.get(this.orderCols[byColumnIndex].colMeta.getColIndex());
if (compareObject(left, right, this.orderCols[byColumnIndex]) <= 0) {
if (compareObject(left, right, this.orderCols[byColumnIndex]) < 0) {
if (this.orderCols[byColumnIndex].orderType == OrderCol.COL_ORDER_TYPE_ASC) {
resultTemp[pr++] = array[p1++];
} else {
resultTemp[pr++] = array[p2++];
}
} else { // if this field is equal, try next
compare(byColumnIndex + 1);
}
} else {
if (this.orderCols[byColumnIndex].orderType == OrderCol.COL_ORDER_TYPE_ASC) {
resultTemp[pr++] = array[p2++];
} else {
resultTemp[pr++] = array[p1++];
}
}
}
public static int compareObject(Object l, Object r, OrderCol orderCol) {
return compareObject((byte[]) l, (byte[]) r, orderCol);
}
public static int compareObject(byte[] left, byte[] right, OrderCol orderCol) {
int colType = orderCol.getColMeta().getColType();
switch (colType) {
case ColMeta.COL_TYPE_DECIMAL:
case ColMeta.COL_TYPE_INT:
case ColMeta.COL_TYPE_SHORT:
case ColMeta.COL_TYPE_LONG:
case ColMeta.COL_TYPE_FLOAT:
case ColMeta.COL_TYPE_DOUBLE:
case ColMeta.COL_TYPE_LONGLONG:
case ColMeta.COL_TYPE_INT24:
case ColMeta.COL_TYPE_NEWDECIMAL:
// treat date type as number
case ColMeta.COL_TYPE_DATE:
case ColMeta.COL_TYPE_TIMSTAMP:
case ColMeta.COL_TYPE_TIME:
case ColMeta.COL_TYPE_YEAR:
case ColMeta.COL_TYPE_DATETIME:
case ColMeta.COL_TYPE_NEWDATE:
case ColMeta.COL_TYPE_BIT:
return ByteUtil.compareNumberByte(left, right);
case ColMeta.COL_TYPE_VAR_STRING:
case ColMeta.COL_TYPE_STRING:
// execute ENUM and SET as string
case ColMeta.COL_TYPE_ENUM:
case ColMeta.COL_TYPE_SET:
return BytesTools.compareTo(left, right);
// ignore BLOB GEOMETRY
default:
break;
}
return 0;
}
}

View File

@@ -1,62 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp.tmp;
import com.actiontech.dble.net.mysql.RowDataPacket;
import java.util.List;
/**
* @author coderczp-2014-12-17
*/
public interface HeapItf {
/**
* buildHeap
*/
void buildHeap();
/**
* getRoot
*
* @return
*/
RowDataPacket getRoot();
/**
* ADD ITEM
*
* @param row
*/
void add(RowDataPacket row);
/**
* getData
*
* @return
*/
List<RowDataPacket> getData();
/**
* setRoot
*
* @param root
*/
void setRoot(RowDataPacket root);
/**
* addIfRequired
*
* @param row
*/
boolean addIfRequired(RowDataPacket row);
/**
* heapSort
*/
void heapSort(int size);
}

View File

@@ -1,142 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp.tmp;
import com.actiontech.dble.net.mysql.RowDataPacket;
import java.util.ArrayList;
import java.util.List;
/**
* MaxHeap FOR ASC
*
* @author coderczp-2014-12-8
*/
public class MaxHeap implements HeapItf {
private RowDataCmp cmp;
private List<RowDataPacket> data;
public MaxHeap(RowDataCmp cmp, int size) {
this.cmp = cmp;
this.data = new ArrayList<>();
}
@Override
public void buildHeap() {
int len = data.size();
for (int i = len / 2 - 1; i >= 0; i--) {
heapifyRecursive(i, len);
}
}
private void heapify(int i, int size) {
int max = 0;
int mid = size >> 1; // ==size/2
while (i <= mid) {
max = i;
int left = i << 1;
int right = left + 1;
if (left < size && cmp.compare(data.get(left), data.get(i)) > 0) {
max = left;
}
if (right < size && cmp.compare(data.get(right), data.get(max)) > 0) {
max = right;
}
if (i == max) {
break;
}
if (i != max) {
RowDataPacket tmp = data.get(i);
data.set(i, data.get(max));
data.set(max, tmp);
i = max;
}
}
}
protected void heapifyRecursive(int i, int size) {
int l = left(i);
int r = right(i);
int max = i;
if (l < size && cmp.compare(data.get(l), data.get(i)) > 0) {
max = l;
}
if (r < size && cmp.compare(data.get(r), data.get(max)) > 0) {
max = r;
}
if (i == max) {
return;
}
swap(i, max);
heapifyRecursive(max, size);
}
private int right(int i) {
return (i + 1) << 1;
}
private int left(int i) {
return ((i + 1) << 1) - 1;
}
private void swap(int i, int j) {
RowDataPacket tmp = data.get(i);
RowDataPacket elementAt = data.get(j);
data.set(i, elementAt);
data.set(j, tmp);
}
@Override
public RowDataPacket getRoot() {
return data.get(0);
}
@Override
public void setRoot(RowDataPacket root) {
data.set(0, root);
heapifyRecursive(0, data.size());
}
@Override
public List<RowDataPacket> getData() {
return data;
}
@Override
public void add(RowDataPacket row) {
data.add(row);
}
@Override
public boolean addIfRequired(RowDataPacket row) {
RowDataPacket root = getRoot();
// remove the smallest
if (cmp.compare(row, root) < 0) {
setRoot(row);
return true;
}
return false;
}
@Override
public void heapSort(int size) {
final int total = data.size();
if (size <= 0 || size > total) {
size = total;
}
final int min = size == total ? 0 : (total - size - 1);
// change the tail and head
for (int i = total - 1; i > min; i--) {
swap(0, i);
heapifyRecursive(0, i);
}
}
}

View File

@@ -1,109 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp.tmp;
import com.actiontech.dble.net.mysql.RowDataPacket;
import java.util.ArrayList;
import java.util.List;
/**
* MinHeap for DESC
*
* @author coderczp-2014-12-8
*/
public class MinHeap implements HeapItf {
private RowDataCmp cmp;
private List<RowDataPacket> data;
public MinHeap(RowDataCmp cmp, int size) {
this.cmp = cmp;
this.data = new ArrayList<>();
}
@Override
public void buildHeap() {
int len = data.size();
for (int i = len / 2 - 1; i >= 0; i--) {
heapify(i, len);
}
}
private void heapify(int i, int size) {
int l = left(i);
int r = right(i);
int smallest = i;
if (l < size && cmp.compare(data.get(l), data.get(i)) < 0) {
smallest = l;
}
if (r < size && cmp.compare(data.get(r), data.get(smallest)) < 0) {
smallest = r;
}
if (i == smallest) {
return;
}
swap(i, smallest);
heapify(smallest, size);
}
private int right(int i) {
return (i + 1) << 1;
}
private int left(int i) {
return ((i + 1) << 1) - 1;
}
private void swap(int i, int j) {
RowDataPacket tmp = data.get(i);
RowDataPacket elementAt = data.get(j);
data.set(i, elementAt);
data.set(j, tmp);
}
public RowDataPacket getRoot() {
return data.get(0);
}
public void setRoot(RowDataPacket root) {
data.set(0, root);
heapify(0, data.size());
}
public List<RowDataPacket> getData() {
return data;
}
public void add(RowDataPacket row) {
data.add(row);
}
@Override
public boolean addIfRequired(RowDataPacket row) {
RowDataPacket root = getRoot();
if (cmp.compare(row, root) > 0) {
setRoot(row);
return true;
}
return false;
}
@Override
public void heapSort(int size) {
final int total = data.size();
if (size <= 0 || size > total) {
size = total;
}
final int min = size == total ? 0 : (total - size - 1);
for (int i = total - 1; i > min; i--) {
swap(0, i);
heapify(0, i);
}
}
}

View File

@@ -1,47 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp.tmp;
import com.actiontech.dble.net.mysql.RowDataPacket;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import com.actiontech.dble.sqlengine.mpp.RowDataPacketSorter;
import java.util.Comparator;
/**
* @author coderczp-2014-12-8
*/
public class RowDataCmp implements Comparator<RowDataPacket> {
private OrderCol[] orderCols;
public RowDataCmp(OrderCol[] orderCols) {
this.orderCols = orderCols;
}
@Override
public int compare(RowDataPacket o1, RowDataPacket o2) {
OrderCol[] tmp = this.orderCols;
int cmp = 0;
//compare the columns of order by
int type = OrderCol.COL_ORDER_TYPE_ASC;
for (OrderCol aTmp : tmp) {
int colIndex = aTmp.colMeta.getColIndex();
byte[] left = o1.fieldValues.get(colIndex);
byte[] right = o2.fieldValues.get(colIndex);
if (aTmp.orderType == type) {
cmp = RowDataPacketSorter.compareObject(left, right, aTmp);
} else {
cmp = RowDataPacketSorter.compareObject(right, left, aTmp);
}
if (cmp != 0) {
return cmp;
}
}
return cmp;
}
}

View File

@@ -1,85 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.sqlengine.mpp.tmp;
import com.actiontech.dble.net.mysql.RowDataPacket;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import com.actiontech.dble.sqlengine.mpp.RowDataPacketSorter;
import java.util.Collections;
import java.util.List;
/**
* @author coderczp-2014-12-8
*/
public class RowDataSorter extends RowDataPacketSorter {
// total(=offset+limit)
private volatile int total;
// record size(=limit)
private volatile int size;
// heap
private volatile HeapItf heap;
// compartor
private volatile RowDataCmp cmp;
// has been buildHeap
private volatile boolean hasBuild;
public RowDataSorter(OrderCol[] orderCols) {
super(orderCols);
this.cmp = new RowDataCmp(orderCols);
}
public synchronized void setLimit(int start, int offset) {
if (start < 0) {
start = 0;
}
if (offset <= 0) {
this.total = this.size = Integer.MAX_VALUE;
} else {
this.total = start + offset;
this.size = offset;
}
// use ASC, comparer will know ASC/DESC
this.heap = new MaxHeap(cmp, total);
}
@Override
public synchronized boolean addRow(RowDataPacket row) {
if (heap.getData().size() < total) {
heap.add(row);
return true;
}
if (heap.getData().size() == total && !hasBuild) {
heap.buildHeap();
hasBuild = true;
}
return heap.addIfRequired(row);
}
@Override
public List<RowDataPacket> getSortedResult() {
final List<RowDataPacket> data = heap.getData();
if (data.size() < 2) {
return data;
}
if (total - size > data.size()) {
return Collections.emptyList();
}
if (!hasBuild) {
heap.buildHeap();
}
heap.heapSort(this.size);
return heap.getData();
}
public RowDataCmp getCmp() {
return cmp;
}
}

View File

@@ -1,42 +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 com.actiontech.dble.memory.unsafe.array;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import org.junit.Assert;
import org.junit.Test;
public class LongArraySuite {
@Test
public void basicTest() {
long[] bytes = new long[2];
LongArray arr = new LongArray(MemoryBlock.fromLongArray(bytes));
arr.set(0, 1L);
arr.set(1, 2L);
arr.set(1, 3L);
Assert.assertEquals(2, arr.size());
Assert.assertEquals(1L, arr.get(0));
Assert.assertEquals(3L, arr.get(1));
arr.zeroOut();
Assert.assertEquals(0L, arr.get(0));
Assert.assertEquals(0L, arr.get(1));
}
}

View File

@@ -1,121 +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 com.actiontech.dble.memory.unsafe.hash;
import com.actiontech.dble.memory.unsafe.Platform;
import org.junit.Assert;
import org.junit.Test;
import java.nio.charset.StandardCharsets;
import java.util.HashSet;
import java.util.Random;
import java.util.Set;
/**
* Test file based on Guava's Murmur3Hash32Test.
*/
public class Murmur3_x86_32Suite {
private static final Murmur3OfX86And32Bit hasher = new Murmur3OfX86And32Bit(0);
@Test
public void testKnownIntegerInputs() {
Assert.assertEquals(593689054, hasher.hashInt(0));
Assert.assertEquals(-189366624, hasher.hashInt(-42));
Assert.assertEquals(-1134849565, hasher.hashInt(42));
Assert.assertEquals(-1718298732, hasher.hashInt(Integer.MIN_VALUE));
Assert.assertEquals(-1653689534, hasher.hashInt(Integer.MAX_VALUE));
}
@Test
public void testKnownLongInputs() {
Assert.assertEquals(1669671676, hasher.hashLong(0L));
Assert.assertEquals(-846261623, hasher.hashLong(-42L));
Assert.assertEquals(1871679806, hasher.hashLong(42L));
Assert.assertEquals(1366273829, hasher.hashLong(Long.MIN_VALUE));
Assert.assertEquals(-2106506049, hasher.hashLong(Long.MAX_VALUE));
}
@Test
public void randomizedStressTest() {
int size = 65536;
Random rand = new Random();
// A set used to track collision rate.
Set<Integer> hashcodes = new HashSet<Integer>();
for (int i = 0; i < size; i++) {
int vint = rand.nextInt();
long lint = rand.nextLong();
Assert.assertEquals(hasher.hashInt(vint), hasher.hashInt(vint));
Assert.assertEquals(hasher.hashLong(lint), hasher.hashLong(lint));
hashcodes.add(hasher.hashLong(lint));
}
// A very loose bound.
Assert.assertTrue(hashcodes.size() > size * 0.95);
}
@Test
public void randomizedStressTestBytes() {
int size = 65536;
Random rand = new Random();
// A set used to track collision rate.
Set<Integer> hashcodes = new HashSet<Integer>();
for (int i = 0; i < size; i++) {
int byteArrSize = rand.nextInt(100) * 8;
byte[] bytes = new byte[byteArrSize];
rand.nextBytes(bytes);
Assert.assertEquals(
hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize),
hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
hashcodes.add(hasher.hashUnsafeWords(
bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
}
// A very loose bound.
Assert.assertTrue(hashcodes.size() > size * 0.95);
}
@Test
public void randomizedStressTestPaddedStrings() {
int size = 64000;
// A set used to track collision rate.
Set<Integer> hashcodes = new HashSet<Integer>();
for (int i = 0; i < size; i++) {
int byteArrSize = 8;
byte[] strBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8);
byte[] paddedBytes = new byte[byteArrSize];
System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length);
Assert.assertEquals(
hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize),
hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
hashcodes.add(hasher.hashUnsafeWords(
paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
}
// A very loose bound.
Assert.assertTrue(hashcodes.size() > size * 0.95);
}
}

View File

@@ -1,639 +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 com.actiontech.dble.memory.unsafe.map;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.array.ByteArrayMethods;
import com.actiontech.dble.memory.unsafe.memory.TestMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.storage.DataNodeDiskManager;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
public abstract class AbstractBytesToBytesMapSuite {
private final Random rand = new Random(42);
ServerPropertyConf conf = new ServerPropertyConf()
.set("server.memory.offHeap.enabled", "" + useOffHeapMemoryAllocator())
.set("server.memory.offHeap.size", "256mb");
private TestMemoryManager memoryManager =
new TestMemoryManager(conf
);
private DataNodeMemoryManager dataNodeMemoryManager =
new DataNodeMemoryManager(memoryManager, 0);
private SerializerManager serializerManager = new SerializerManager();
private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes
final LinkedList<File> spillFilesCreated = new LinkedList<File>();
File tempDir;
DataNodeDiskManager blockManager = new DataNodeDiskManager(conf, true);
/*
private static final class CompressStream extends AbstractFunction1<OutputStream, OutputStream> {
@Override
public OutputStream apply(OutputStream stream) {
return stream;
}
}
*/
@Before
public void setup() {
}
@After
public void tearDown() throws IOException {
//Utils.deleteRecursively(tempDir);
//tempDir = null;
if (dataNodeMemoryManager != null) {
Assert.assertEquals(0L, dataNodeMemoryManager.cleanUpAllAllocatedMemory());
long leakedMemory = dataNodeMemoryManager.getMemoryConsumptionForThisConnection();
dataNodeMemoryManager = null;
Assert.assertEquals(0L, leakedMemory);
}
}
protected abstract boolean useOffHeapMemoryAllocator();
private static byte[] getByteArray(Object base, long offset, int size) {
final byte[] arr = new byte[size];
Platform.copyMemory(base, offset, arr, Platform.BYTE_ARRAY_OFFSET, size);
return arr;
}
private byte[] getRandomByteArray(int numWords) {
Assert.assertTrue(numWords >= 0);
final int lengthInBytes = numWords * 8;
final byte[] bytes = new byte[lengthInBytes];
rand.nextBytes(bytes);
return bytes;
}
/**
* Fast equality checking for byte arrays, since these comparisons are a bottleneck
* in our stress tests.
*/
private static boolean arrayEquals(
byte[] expected,
Object base,
long offset,
long actualLengthBytes) {
return (actualLengthBytes == expected.length) && ByteArrayMethods.arrayEquals(
expected,
Platform.BYTE_ARRAY_OFFSET,
base,
offset,
expected.length
);
}
@Test
public void emptyMap() {
BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 64, PAGE_SIZE_BYTES);
try {
assertEquals(0, map.numKeys());
final int keyLengthInWords = 10;
final int keyLengthInBytes = keyLengthInWords * 8;
final byte[] key = getRandomByteArray(keyLengthInWords);
Assert.assertFalse(map.lookup(key, Platform.BYTE_ARRAY_OFFSET, keyLengthInBytes).isDefined());
Assert.assertFalse(map.iterator().hasNext());
} finally {
map.free();
}
}
@Test
public void setAndRetrieveAKey() {
BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 64, PAGE_SIZE_BYTES);
final int recordLengthWords = 10;
final int recordLengthBytes = recordLengthWords * 8;
final byte[] keyData = getRandomByteArray(recordLengthWords);
final byte[] valueData = getRandomByteArray(recordLengthWords);
try {
final BytesToBytesMap.Location loc =
map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.append(
keyData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
valueData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes
));
// After storing the key and value, the other location methods should return results that
// reflect the result of this store without us having to call lookup() again on the same key.
assertEquals(recordLengthBytes, loc.getKeyLength());
assertEquals(recordLengthBytes, loc.getValueLength());
Assert.assertArrayEquals(keyData,
getByteArray(loc.getKeyBase(), loc.getKeyOffset(), recordLengthBytes));
Assert.assertArrayEquals(valueData,
getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes));
// After calling lookup() the location should still point to the correct data.
Assert.assertTrue(
map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes).isDefined());
assertEquals(recordLengthBytes, loc.getKeyLength());
assertEquals(recordLengthBytes, loc.getValueLength());
Assert.assertArrayEquals(keyData,
getByteArray(loc.getKeyBase(), loc.getKeyOffset(), recordLengthBytes));
Assert.assertArrayEquals(valueData,
getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes));
try {
Assert.assertTrue(loc.append(
keyData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
valueData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes
));
Assert.fail("Should not be able to set a new value for a key");
} catch (AssertionError e) {
// Expected exception; do nothing.
}
} finally {
map.free();
}
}
private void iteratorTestBase(boolean destructive) throws Exception {
final int size = 4096;
BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, size / 2, PAGE_SIZE_BYTES);
try {
for (long i = 0; i < size; i++) {
final long[] value = new long[]{i};
final BytesToBytesMap.Location loc =
map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8);
Assert.assertFalse(loc.isDefined());
// Ensure that we store some zero-length keys
if (i % 5 == 0) {
Assert.assertTrue(loc.append(
null,
Platform.LONG_ARRAY_OFFSET,
0,
value,
Platform.LONG_ARRAY_OFFSET,
8
));
} else {
Assert.assertTrue(loc.append(
value,
Platform.LONG_ARRAY_OFFSET,
8,
value,
Platform.LONG_ARRAY_OFFSET,
8
));
}
}
final BitSet valuesSeen = new BitSet(size);
final Iterator<BytesToBytesMap.Location> iter;
if (destructive) {
iter = map.destructiveIterator();
} else {
iter = map.iterator();
}
int numPages = map.getNumDataPages();
int countFreedPages = 0;
while (iter.hasNext()) {
final BytesToBytesMap.Location loc = iter.next();
Assert.assertTrue(loc.isDefined());
final long value = Platform.getLong(loc.getValueBase(), loc.getValueOffset());
final long keyLength = loc.getKeyLength();
if (keyLength == 0) {
Assert.assertTrue("value " + value + " was not divisible by 5", value % 5 == 0);
} else {
final long key = Platform.getLong(loc.getKeyBase(), loc.getKeyOffset());
Assert.assertEquals(value, key);
}
valuesSeen.set((int) value);
if (destructive) {
// The iterator moves onto next page and frees previous page
if (map.getNumDataPages() < numPages) {
numPages = map.getNumDataPages();
countFreedPages++;
}
}
}
if (destructive) {
// Latest page is not freed by iterator but by map itself
Assert.assertEquals(countFreedPages, numPages - 1);
}
Assert.assertEquals(size, valuesSeen.cardinality());
} finally {
map.free();
}
}
@Test
public void iteratorTest() throws Exception {
iteratorTestBase(false);
}
@Test
public void destructiveIteratorTest() throws Exception {
iteratorTestBase(true);
}
@Test
public void iteratingOverDataPagesWithWastedSpace() throws Exception {
final int NUM_ENTRIES = 1000 * 1000;
final int KEY_LENGTH = 24;
final int VALUE_LENGTH = 40;
final BytesToBytesMap map =
new BytesToBytesMap(dataNodeMemoryManager, NUM_ENTRIES, PAGE_SIZE_BYTES);
// Each record will take 8 + 24 + 40 = 72 bytes of space in the data page. Our 64-megabyte
// pages won't be evenly-divisible by records of this size, which will cause us to waste some
// space at the end of the page. This is necessary in order for us to take the end-of-record
// handling branch in iterator().
try {
for (int i = 0; i < NUM_ENTRIES; i++) {
final long[] key = new long[]{i, i, i}; // 3 * 8 = 24 bytes
final long[] value = new long[]{i, i, i, i, i}; // 5 * 8 = 40 bytes
final BytesToBytesMap.Location loc = map.lookup(
key,
Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.append(
key,
Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH,
value,
Platform.LONG_ARRAY_OFFSET,
VALUE_LENGTH
));
}
assertEquals(2, map.getNumDataPages());
final BitSet valuesSeen = new BitSet(NUM_ENTRIES);
final Iterator<BytesToBytesMap.Location> iter = map.iterator();
final long[] key = new long[KEY_LENGTH / 8];
final long[] value = new long[VALUE_LENGTH / 8];
while (iter.hasNext()) {
final BytesToBytesMap.Location loc = iter.next();
Assert.assertTrue(loc.isDefined());
assertEquals(KEY_LENGTH, loc.getKeyLength());
assertEquals(VALUE_LENGTH, loc.getValueLength());
Platform.copyMemory(
loc.getKeyBase(),
loc.getKeyOffset(),
key,
Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH
);
Platform.copyMemory(
loc.getValueBase(),
loc.getValueOffset(),
value,
Platform.LONG_ARRAY_OFFSET,
VALUE_LENGTH
);
for (long j : key) {
Assert.assertEquals(key[0], j);
}
for (long j : value) {
Assert.assertEquals(key[0], j);
}
valuesSeen.set((int) key[0]);
}
Assert.assertEquals(NUM_ENTRIES, valuesSeen.cardinality());
} finally {
map.free();
}
}
@Test
public void randomizedStressTest() {
final int size = 65536;
// Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays
// into ByteBuffers in order to use them as keys here.
final Map<ByteBuffer, byte[]> expected = new HashMap<ByteBuffer, byte[]>();
final BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, size, PAGE_SIZE_BYTES);
try {
// Fill the map to 90% full so that we can trigger probing
for (int i = 0; i < size * 0.9; i++) {
final byte[] key = getRandomByteArray(rand.nextInt(10) + 1);
final byte[] value = getRandomByteArray(rand.nextInt(10) + 1);
if (!expected.containsKey(ByteBuffer.wrap(key))) {
expected.put(ByteBuffer.wrap(key), value);
final BytesToBytesMap.Location loc = map.lookup(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.append(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length,
value,
Platform.BYTE_ARRAY_OFFSET,
value.length
));
// After calling putNewKey, the following should be true, even before calling
// lookup():
Assert.assertTrue(loc.isDefined());
assertEquals(key.length, loc.getKeyLength());
assertEquals(value.length, loc.getValueLength());
Assert.assertTrue(arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), key.length));
Assert.assertTrue(
arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), value.length));
}
}
/**
for (Map.Entry<ByteBuffer, byte[]> entry : expected.entrySet()) {
final byte[] key = JavaUtils.bufferToArray(entry.getKey());
final byte[] value = entry.getValue();
final BytesToBytesMap.Location loc =
map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length);
Assert.assertTrue(loc.isDefined());
Assert.assertTrue(
arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), loc.getKeyLength()));
Assert.assertTrue(
arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), loc.getValueLength()));
}
*/
} finally {
map.free();
}
}
@Test
public void randomizedTestWithRecordsLargerThanPageSize() {
final long pageSizeBytes = 128;
final BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 64, pageSizeBytes);
// Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays
// into ByteBuffers in order to use them as keys here.
final Map<ByteBuffer, byte[]> expected = new HashMap<ByteBuffer, byte[]>();
try {
for (int i = 0; i < 1000; i++) {
final byte[] key = getRandomByteArray(rand.nextInt(128));
final byte[] value = getRandomByteArray(rand.nextInt(128));
if (!expected.containsKey(ByteBuffer.wrap(key))) {
expected.put(ByteBuffer.wrap(key), value);
final BytesToBytesMap.Location loc = map.lookup(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.append(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length,
value,
Platform.BYTE_ARRAY_OFFSET,
value.length
));
// After calling putNewKey, the following should be true, even before calling
// lookup():
Assert.assertTrue(loc.isDefined());
assertEquals(key.length, loc.getKeyLength());
assertEquals(value.length, loc.getValueLength());
Assert.assertTrue(arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), key.length));
Assert.assertTrue(
arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), value.length));
}
}
/**
for (Map.Entry<ByteBuffer, byte[]> entry : expected.entrySet()) {
final byte[] key = JavaUtils.bufferToArray(entry.getKey());
final byte[] value = entry.getValue();
final BytesToBytesMap.Location loc =
map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length);
Assert.assertTrue(loc.isDefined());
Assert.assertTrue(
arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), loc.getKeyLength()));
Assert.assertTrue(
arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), loc.getValueLength()));
}
*/
} finally {
map.free();
}
}
@Test
public void failureToAllocateFirstPage() {
memoryManager.limit(1024); // longArray
BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 1, PAGE_SIZE_BYTES);
try {
final long[] emptyArray = new long[0];
final BytesToBytesMap.Location loc =
map.lookup(emptyArray, Platform.LONG_ARRAY_OFFSET, 0);
Assert.assertFalse(loc.isDefined());
Assert.assertFalse(loc.append(
emptyArray, Platform.LONG_ARRAY_OFFSET, 0, emptyArray, Platform.LONG_ARRAY_OFFSET, 0));
} finally {
map.free();
}
}
@Test
public void failureToGrow() {
BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 1, 1024);
try {
boolean success = true;
int i;
for (i = 0; i < 127; i++) {
if (i > 0) {
memoryManager.limit(0);
}
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
success =
loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
if (!success) {
break;
}
}
Assert.assertThat(i, greaterThan(0));
Assert.assertFalse(success);
} finally {
map.free();
}
}
@Test
public void spillInIterator() throws IOException {
BytesToBytesMap map = new BytesToBytesMap(
dataNodeMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false);
try {
int i;
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
}
BytesToBytesMap.MapIterator iter = map.iterator();
for (i = 0; i < 100; i++) {
iter.next();
}
// Non-destructive iterator is not spillable
assertEquals(0, iter.spill(1024L * 10));
for (i = 100; i < 1024; i++) {
iter.next();
}
BytesToBytesMap.MapIterator iter2 = map.destructiveIterator();
for (i = 0; i < 100; i++) {
iter2.next();
}
Assert.assertTrue(iter2.spill(1024) >= 1024);
for (i = 100; i < 1024; i++) {
iter2.next();
}
assertFalse(iter2.hasNext());
} finally {
map.free();
for (File spillFile : spillFilesCreated) {
assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up",
spillFile.exists());
}
}
}
@Test
public void multipleValuesForSameKey() {
BytesToBytesMap map =
new BytesToBytesMap(dataNodeMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false);
try {
int i;
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8)
.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
}
assert map.numKeys() == 1024;
assert map.numValues() == 1024;
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8)
.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
}
assert map.numKeys() == 1024;
assert map.numValues() == 2048;
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
assert loc.isDefined();
assert loc.nextValue();
assert !loc.nextValue();
}
BytesToBytesMap.MapIterator iter = map.iterator();
for (i = 0; i < 2048; i++) {
assert iter.hasNext();
final BytesToBytesMap.Location loc = iter.next();
assert loc.isDefined();
}
} finally {
map.free();
}
}
@Test
public void initialCapacityBoundsChecking() {
try {
new BytesToBytesMap(dataNodeMemoryManager, 0, PAGE_SIZE_BYTES);
Assert.fail("Expected IllegalArgumentException to be thrown");
} catch (IllegalArgumentException e) {
// expected exception
}
try {
new BytesToBytesMap(
dataNodeMemoryManager,
BytesToBytesMap.MAX_CAPACITY + 1,
PAGE_SIZE_BYTES);
Assert.fail("Expected IllegalArgumentException to be thrown");
} catch (IllegalArgumentException e) {
// expected exception
}
}
@Test
public void testPeakMemoryUsed() {
final long recordLengthBytes = 32;
final long pageSizeBytes = 256 + 8; // 8 bytes for end-of-page marker
final long numRecordsPerPage = (pageSizeBytes - 8) / recordLengthBytes;
final BytesToBytesMap map = new BytesToBytesMap(dataNodeMemoryManager, 1024, pageSizeBytes);
// Since BytesToBytesMap is append-only, we expect the total memory consumption to be
// monotonically increasing. More specifically, every time we allocate a new page it
// should increase by exactly the size of the page. In this regard, the memory usage
// at any given time is also the peak memory used.
long previousPeakMemory = map.getPeakMemoryUsedBytes();
long newPeakMemory;
try {
for (long i = 0; i < numRecordsPerPage * 10; i++) {
final long[] value = new long[]{i};
map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append(
value,
Platform.LONG_ARRAY_OFFSET,
8,
value,
Platform.LONG_ARRAY_OFFSET,
8);
newPeakMemory = map.getPeakMemoryUsedBytes();
if (i % numRecordsPerPage == 0) {
// We allocated a new page for this record, so peak memory should change
assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory);
} else {
assertEquals(previousPeakMemory, newPeakMemory);
}
previousPeakMemory = newPeakMemory;
}
// Freeing the map should not change the peak memory
map.free();
newPeakMemory = map.getPeakMemoryUsedBytes();
assertEquals(previousPeakMemory, newPeakMemory);
} finally {
map.free();
}
}
}

View File

@@ -1,26 +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 com.actiontech.dble.memory.unsafe.map;
public class BytesToBytesMapOffHeapSuite extends AbstractBytesToBytesMapSuite {
@Override
protected boolean useOffHeapMemoryAllocator() {
return true;
}
}

View File

@@ -1,26 +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 com.actiontech.dble.memory.unsafe.map;
public class BytesToBytesMapOnHeapSuite extends AbstractBytesToBytesMapSuite {
@Override
protected boolean useOffHeapMemoryAllocator() {
return false;
}
}

View File

@@ -1,307 +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 com.actiontech.dble.memory.unsafe.map;
import com.actiontech.dble.memory.SeverMemory;
import com.actiontech.dble.memory.unsafe.KVIterator;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryManager;
import com.actiontech.dble.memory.unsafe.row.BufferHolder;
import com.actiontech.dble.memory.unsafe.row.StructType;
import com.actiontech.dble.memory.unsafe.row.UnsafeRow;
import com.actiontech.dble.memory.unsafe.row.UnsafeRowWriter;
import com.actiontech.dble.memory.unsafe.utils.BytesTools;
import com.actiontech.dble.sqlengine.mpp.ColMeta;
import com.actiontech.dble.sqlengine.mpp.OrderCol;
import org.apache.log4j.Logger;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.*;
/**
* Created by zagnix on 2016/6/4.
*/
public class UnsafeFixedWidthAggregationMapSuite {
private StructType groupKeySchema;
private StructType aggBufferSchema;
private UnsafeRow emptyAggregationBuffer;
private long PAGE_SIZE_BYTES = 1L << 20;
private final Random rand = new Random(42);
private static Logger LOGGER = Logger.getLogger(UnsafeFixedWidthAggregationMapSuite.class);
@Test
public void testAggregateMap() throws NoSuchFieldException, IllegalAccessException, IOException {
SeverMemory severMemory = new SeverMemory();
MemoryManager memoryManager = severMemory.getResultMergeMemoryManager();
DataNodeMemoryManager dataNodeMemoryManager = new DataNodeMemoryManager(memoryManager, Thread.currentThread().getId());
/**
* group key
*/
int fieldCount = 2;
ColMeta colMeta = null;
Map<String, ColMeta> colMetaMap = new HashMap<String, ColMeta>(fieldCount);
colMeta = new ColMeta(0, ColMeta.COL_TYPE_STRING);
colMetaMap.put("id", colMeta);
colMeta = new ColMeta(1, ColMeta.COL_TYPE_STRING);
colMetaMap.put("name", colMeta);
OrderCol[] orderCols = new OrderCol[1];
OrderCol orderCol = new OrderCol(colMetaMap.get("id"), OrderCol.COL_ORDER_TYPE_DESC);
orderCols[0] = orderCol;
groupKeySchema = new StructType(colMetaMap, fieldCount);
groupKeySchema.setOrderCols(orderCols);
/**
* value key
*/
fieldCount = 4;
colMeta = null;
colMetaMap = new HashMap<String, ColMeta>(fieldCount);
colMeta = new ColMeta(0, ColMeta.COL_TYPE_STRING);
colMetaMap.put("id", colMeta);
colMeta = new ColMeta(1, ColMeta.COL_TYPE_STRING);
colMetaMap.put("name", colMeta);
colMeta = new ColMeta(2, ColMeta.COL_TYPE_INT);
colMetaMap.put("age", colMeta);
colMeta = new ColMeta(3, ColMeta.COL_TYPE_LONGLONG);
colMetaMap.put("score", colMeta);
orderCols = new OrderCol[1];
orderCol = new OrderCol(colMetaMap.get("id"), OrderCol.COL_ORDER_TYPE_DESC);
orderCols[0] = orderCol;
aggBufferSchema = new StructType(colMetaMap, fieldCount);
aggBufferSchema.setOrderCols(orderCols);
/**
*emtpy Row value
*/
BufferHolder bufferHolder;
emptyAggregationBuffer = new UnsafeRow(4);
bufferHolder = new BufferHolder(emptyAggregationBuffer, 0);
UnsafeRowWriter unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 4);
bufferHolder.reset();
String value = "o";
unsafeRowWriter.write(0, value.getBytes());
unsafeRowWriter.write(1, value.getBytes());
emptyAggregationBuffer.setInt(2, 0);
emptyAggregationBuffer.setLong(3, 0);
emptyAggregationBuffer.setTotalSize(bufferHolder.totalSize());
UnsafeFixedWidthAggregationMap map = new UnsafeFixedWidthAggregationMap(
emptyAggregationBuffer,
aggBufferSchema,
groupKeySchema,
dataNodeMemoryManager,
2 * 1024,
PAGE_SIZE_BYTES,
true);
int i;
List<UnsafeRow> rows = new ArrayList<UnsafeRow>();
for (i = 0; i < 100000; i++) {
/**
* key
*/
UnsafeRow groupKey = new UnsafeRow(2);
bufferHolder = new BufferHolder(groupKey, 0);
unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 2);
bufferHolder.reset();
unsafeRowWriter.write(0, BytesTools.toBytes(rand.nextInt(10000000)));
unsafeRowWriter.write(1, BytesTools.toBytes(rand.nextInt(10000000)));
groupKey.setTotalSize(bufferHolder.totalSize());
UnsafeRow valueKey = new UnsafeRow(4);
bufferHolder = new BufferHolder(valueKey, 0);
unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 4);
bufferHolder.reset();
unsafeRowWriter.write(0, BytesTools.toBytes(rand.nextInt(10)));
unsafeRowWriter.write(1, BytesTools.toBytes(rand.nextInt(10)));
valueKey.setInt(2, i);
valueKey.setLong(3, 1);
valueKey.setTotalSize(bufferHolder.totalSize());
if (map.find(groupKey)) {
UnsafeRow rs = map.getAggregationBuffer(groupKey);
rs.setLong(3, i + valueKey.getLong(3));
rs.setInt(2, 100 + valueKey.getInt(2));
} else {
map.put(groupKey, valueKey);
}
rows.add(valueKey);
}
KVIterator<UnsafeRow, UnsafeRow> iter = map.iterator();
int j = 0;
while (iter.next()) {
Assert.assertEquals(j, iter.getValue().getInt(2));
j++;
iter.getValue().setInt(2, 5000000);
iter.getValue().setLong(3, 600000);
}
Assert.assertEquals(rows.size(), j);
int k = 0;
KVIterator<UnsafeRow, UnsafeRow> iter1 = map.iterator();
while (iter1.next()) {
k++;
// LOGGER.error("(" + BytesTools.toInt(iter1.getKey().getBinary(0)) + "," +
// iter1.getValue().getInt(2) +"," +iter1.getValue().getLong(3)+")");
Assert.assertEquals(5000000, iter1.getValue().getInt(2));
Assert.assertEquals(600000, iter1.getValue().getLong(3));
}
Assert.assertEquals(j, k);
map.free();
}
@Test
public void testWithMemoryLeakDetection() throws IOException, NoSuchFieldException, IllegalAccessException {
SeverMemory severMemory = new SeverMemory();
MemoryManager memoryManager = severMemory.getResultMergeMemoryManager();
DataNodeMemoryManager dataNodeMemoryManager = new DataNodeMemoryManager(memoryManager,
Thread.currentThread().getId());
int fieldCount = 3;
ColMeta colMeta = null;
Map<String, ColMeta> colMetaMap = new HashMap<String, ColMeta>(fieldCount);
colMeta = new ColMeta(0, ColMeta.COL_TYPE_STRING);
colMetaMap.put("id", colMeta);
colMeta = new ColMeta(1, ColMeta.COL_TYPE_STRING);
colMetaMap.put("name", colMeta);
colMeta = new ColMeta(2, ColMeta.COL_TYPE_STRING);
colMetaMap.put("age", colMeta);
OrderCol[] orderCols = new OrderCol[1];
OrderCol orderCol = new OrderCol(colMetaMap.get("id"), OrderCol.COL_ORDER_TYPE_DESC);
orderCols[0] = orderCol;
groupKeySchema = new StructType(colMetaMap, fieldCount);
groupKeySchema.setOrderCols(orderCols);
fieldCount = 3;
colMeta = null;
colMetaMap = new HashMap<String, ColMeta>(fieldCount);
colMeta = new ColMeta(0, ColMeta.COL_TYPE_LONGLONG);
colMetaMap.put("age", colMeta);
colMeta = new ColMeta(1, ColMeta.COL_TYPE_LONGLONG);
colMetaMap.put("age1", colMeta);
colMeta = new ColMeta(2, ColMeta.COL_TYPE_STRING);
colMetaMap.put("name", colMeta);
orderCols = new OrderCol[1];
orderCol = new OrderCol(colMetaMap.get("id"), OrderCol.COL_ORDER_TYPE_DESC);
orderCols[0] = orderCol;
aggBufferSchema = new StructType(colMetaMap, fieldCount);
aggBufferSchema.setOrderCols(orderCols);
/**
* value
*/
BufferHolder bufferHolder;
emptyAggregationBuffer = new UnsafeRow(3);
bufferHolder = new BufferHolder(emptyAggregationBuffer, 0);
UnsafeRowWriter unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 3);
bufferHolder.reset();
String value = "ok,hello";
emptyAggregationBuffer.setLong(0, 0);
emptyAggregationBuffer.setLong(1, 0);
unsafeRowWriter.write(2, value.getBytes());
emptyAggregationBuffer.setTotalSize(bufferHolder.totalSize());
UnsafeFixedWidthAggregationMap map = new UnsafeFixedWidthAggregationMap(
emptyAggregationBuffer,
aggBufferSchema,
groupKeySchema,
dataNodeMemoryManager,
2 * 1024,
PAGE_SIZE_BYTES,
false
);
int i;
List<UnsafeRow> rows = new ArrayList<UnsafeRow>();
for (i = 0; i < 1000; i++) {
String line = "testUnsafeRow" + i;
/**
* key
*/
UnsafeRow groupKey = new UnsafeRow(3);
bufferHolder = new BufferHolder(groupKey, 0);
unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 3);
bufferHolder.reset();
final byte[] key = getRandomByteArray(rand.nextInt(8));
String age = "5" + i;
unsafeRowWriter.write(0, key);
unsafeRowWriter.write(1, line.getBytes());
unsafeRowWriter.write(2, age.getBytes());
groupKey.setTotalSize(bufferHolder.totalSize());
map.getAggregationBuffer(groupKey);
rows.add(groupKey);
}
Assert.assertEquals(i, rows.size());
UnsafeRow row = rows.get(12);
UnsafeRow rs = map.getAggregationBuffer(row);
rs.setLong(0, 12);
rs = map.getAggregationBuffer(row);
Assert.assertEquals(12, rs.getLong(0));
map.free();
}
private byte[] getRandomByteArray(int numWords) {
Assert.assertTrue(numWords >= 0);
final int lengthInBytes = numWords * 8;
final byte[] bytes = new byte[lengthInBytes];
rand.nextBytes(bytes);
return bytes;
}
}

View File

@@ -1,12 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory;
/**
* Created by zagnix on 2016/6/6.
*/
public interface MemoryManagerSuite {
}

View File

@@ -1,120 +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 com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryMode;
import com.actiontech.dble.memory.unsafe.memory.mm.ResultMergeMemoryManager;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import org.junit.Assert;
import org.junit.Test;
public class TaskMemoryManagerSuite {
@Test
public void leakedPageMemoryIsDetected() {
final DataNodeMemoryManager manager = new DataNodeMemoryManager(
new ResultMergeMemoryManager(
new ServerPropertyConf().set("server.memory.offHeap.enabled", "false")
.set("server.memory.offHeap.size", "32768"),
1,
Long.MAX_VALUE
),
0);
manager.allocatePage(4096, null); // leak memory
Assert.assertEquals(4096, manager.getMemoryConsumptionForThisConnection());
Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory());
}
@Test
public void encodePageNumberAndOffsetOffHeap() {
final ServerPropertyConf conf = new ServerPropertyConf()
.set("server.memory.offHeap.enabled", "true")
.set("server.memory.offHeap.size", "1000");
final DataNodeMemoryManager manager = new DataNodeMemoryManager(new TestMemoryManager(conf), 0);
final MemoryBlock dataPage = manager.allocatePage(256, null);
// In off-heap mode, an offset is an absolute address that may require more than 51 bits to
// encode. This map exercises that corner-case:
final long offset = ((1L << DataNodeMemoryManager.OFFSET_BITS) + 10);
final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, offset);
Assert.assertEquals(null, manager.getPage(encodedAddress));
Assert.assertEquals(offset, manager.getOffsetInPage(encodedAddress));
}
@Test
public void encodePageNumberAndOffsetOnHeap() {
final DataNodeMemoryManager manager = new DataNodeMemoryManager(
new TestMemoryManager(new ServerPropertyConf().set("server.memory.offHeap.enabled", "false")), 0);
final MemoryBlock dataPage = manager.allocatePage(256, null);
final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64);
Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress));
Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress));
}
@Test
public void cooperativeSpilling() throws InterruptedException {
final TestMemoryManager memoryManager = new TestMemoryManager(new ServerPropertyConf());
memoryManager.limit(100);
final DataNodeMemoryManager manager = new DataNodeMemoryManager(memoryManager, 0);
TestMemoryConsumer c1 = new TestMemoryConsumer(manager);
TestMemoryConsumer c2 = new TestMemoryConsumer(manager);
c1.use(100);
Assert.assertEquals(100, c1.getUsed());
c2.use(100);
Assert.assertEquals(100, c2.getUsed());
Assert.assertEquals(0, c1.getUsed()); // spilled
c1.use(100);
Assert.assertEquals(100, c1.getUsed());
Assert.assertEquals(0, c2.getUsed()); // spilled
c1.use(50);
Assert.assertEquals(50, c1.getUsed()); // spilled
Assert.assertEquals(0, c2.getUsed());
c2.use(50);
Assert.assertEquals(50, c1.getUsed());
Assert.assertEquals(50, c2.getUsed());
c1.use(100);
Assert.assertEquals(100, c1.getUsed());
Assert.assertEquals(0, c2.getUsed()); // spilled
c1.free(20);
Assert.assertEquals(80, c1.getUsed());
c2.use(10);
Assert.assertEquals(80, c1.getUsed());
Assert.assertEquals(10, c2.getUsed());
c2.use(100);
Assert.assertEquals(100, c2.getUsed());
Assert.assertEquals(0, c1.getUsed()); // spilled
c1.free(0);
c2.free(100);
Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory());
}
@Test
public void offHeapConfigurationBackwardsCompatibility() {
final ServerPropertyConf conf = new ServerPropertyConf()
.set("server.memory.offHeap.enabled", "true")
.set("server.memory.offHeap.size", "1000");
final DataNodeMemoryManager manager = new DataNodeMemoryManager(new TestMemoryManager(conf), 0);
Assert.assertSame(MemoryMode.OFF_HEAP, manager.tungstenMemoryMode);
}
}

View File

@@ -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 com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryConsumer;
import java.io.IOException;
public class TestMemoryConsumer extends MemoryConsumer {
public TestMemoryConsumer(DataNodeMemoryManager memoryManager) {
super(memoryManager);
}
@Override
public long spill(long size, MemoryConsumer trigger) throws IOException {
long used = getUsed();
free(used);
return used;
}
void use(long size) throws InterruptedException {
long got = dataNodeMemoryManager.acquireExecutionMemory(
size,
dataNodeMemoryManager.tungstenMemoryMode,
this);
used += got;
}
void free(long size) {
used -= size;
dataNodeMemoryManager.releaseExecutionMemory(
size,
dataNodeMemoryManager.tungstenMemoryMode,
this);
}
}

View File

@@ -1,58 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.memory;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.MemoryMode;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
public class TestMemoryManager extends MemoryManager {
public TestMemoryManager(ServerPropertyConf conf) {
super(conf, 1, Long.MAX_VALUE);
}
private boolean oomOnce = false;
private long available = Long.MAX_VALUE;
@Override
protected long acquireExecutionMemory(
long numBytes,
long taskAttemptId,
MemoryMode memoryMode) {
if (oomOnce) {
oomOnce = false;
return 0;
} else if (available >= numBytes) {
available -= numBytes;
return numBytes;
} else {
long grant = available;
available = 0;
return grant;
}
}
@Override
public void releaseExecutionMemory(
long numBytes,
long taskAttemptId,
MemoryMode memoryMode) {
available += numBytes;
}
public void markExecutionAsOutOfMemoryOnce() {
oomOnce = true;
}
public void limit(long avail) {
available = avail;
}
}

View File

@@ -1,53 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.row;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
/**
* Created by zagnix on 2016/6/27.
*/
public class UnsafeRowList {
@Test
public void testUnsafeRowList() {
ArrayList<UnsafeRow> list = new ArrayList<UnsafeRow>();
UnsafeRow unsafeRow;
BufferHolder bufferHolder;
UnsafeRowWriter unsafeRowWriter;
String line = "testUnsafeRow";
for (int i = 0; i < 10; i++) {
unsafeRow = new UnsafeRow(3);
bufferHolder = new BufferHolder(unsafeRow);
unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 3);
bufferHolder.reset();
unsafeRow.setInt(0, 89);
unsafeRowWriter.write(1, line.getBytes(), 0, line.length());
unsafeRow.setInt(2, 23);
unsafeRow.setTotalSize(bufferHolder.totalSize());
list.add(unsafeRow);
}
for (int i = 0; i < 10; i++) {
UnsafeRow row = list.get(i);
row.setInt(0, 1000 + i);
}
for (int i = 0; i < 10; i++) {
UnsafeRow row = list.get(i);
Assert.assertEquals(1000 + i, row.getInt(0));
}
}
}

View File

@@ -1,95 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.row;
import junit.framework.Assert;
import org.junit.Test;
import java.math.BigDecimal;
import static org.junit.Assert.assertEquals;
/**
* Created by zagnix on 2016/6/10.
*/
public class UnsafeRowSuite {
@Test
public void testUnsafeRowSingle() {
UnsafeRow unsafeRow = new UnsafeRow(5);
BufferHolder bufferHolder = new BufferHolder(unsafeRow, 64);
UnsafeRowWriter unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 5);
bufferHolder.reset();
String line2 = "testUnsafeRow3";
unsafeRow.setFloat(0, 7.4f);
unsafeRow.setInt(1, 7);
unsafeRow.setLong(2, 455555);
unsafeRowWriter.write(3, line2.getBytes(), 0, line2.length());
unsafeRow.setNullAt(4);
unsafeRow.setInt(1, 9);
assert (unsafeRow.getFloat(0) == 7.4f);
assert (unsafeRow.getInt(1) == 9);
assert (unsafeRow.getLong(2) == 455555);
Assert.assertEquals("testUnsafeRow3", new String(unsafeRow.getBinary(3)));
assert (false == unsafeRow.isNullAt(3));
assert (true == unsafeRow.isNullAt(4));
}
public void testUnsafeRowWithDecimal() {
int fieldCount = 4;
String value = "12345678901234567890123456789.0123456789";
String value1 = "100";
BigDecimal decimal = new BigDecimal(value);
BigDecimal decimal1 = new BigDecimal(value1);
System.out.println("decimal precision : " + decimal.precision() + ", scale : " + decimal.scale());
UnsafeRow unsafeRow = new UnsafeRow(fieldCount);
BufferHolder bufferHolder = new BufferHolder(unsafeRow, 64);
UnsafeRowWriter unsafeRowWriter = new UnsafeRowWriter(bufferHolder, fieldCount);
bufferHolder.reset();
unsafeRow.setInt(0, 100);
unsafeRow.setDouble(1, 0.99);
unsafeRow.setLong(2, 1000);
unsafeRowWriter.write(3, decimal);
assertEquals(100, unsafeRow.getInt(0));
assertEquals("0.99", String.valueOf(unsafeRow.getDouble(1)));
assertEquals(1000, unsafeRow.getLong(2));
assertEquals(decimal, unsafeRow.getDecimal(3, decimal.scale()));
unsafeRow.updateDecimal(3, decimal1);
assertEquals(decimal1, unsafeRow.getDecimal(3, decimal1.scale()));
// update null decimal
BigDecimal nullDecimal = null;
unsafeRow.updateDecimal(3, nullDecimal);
assertEquals(nullDecimal, unsafeRow.getDecimal(3, 0));
unsafeRow.updateDecimal(3, decimal);
assertEquals(decimal, unsafeRow.getDecimal(3, decimal.scale()));
}
// @Test
// public void testUnsafeRowInsert(){
// UnsafeRow unsafeRow = new UnsafeRow(4);
//
// assert(unsafeRow.getFloat(0) == 7.4f);
// assert(unsafeRow.getInt(1) == 9);
// assert(unsafeRow.getLong(2) == 455555);
// Assert.assertEquals("testUnsafeRow3",new String(unsafeRow.getBinary(3)));
// }
};

View File

@@ -1,25 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.sort;
import com.actiontech.dble.memory.unsafe.utils.JavaUtils;
/**
* Created by zagnix on 2016/6/6.
*/
public class HashPartitioner {
private int index = 0;
public HashPartitioner(int i) {
this.index = i;
}
public int getPartition(String key) {
return JavaUtils.nonNegativeMod(key.hashCode(), index);
}
}

View File

@@ -1,135 +0,0 @@
/**
* Copyright 2015 Stijn de Gouw
* <p>
* Licensed 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 com.actiontech.dble.memory.unsafe.sort;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* This codes generates a int array which fails the standard TimSort.
* <p>
* The blog that reported the bug
* http://www.envisage-project.eu/timsort-specification-and-verification/
* <p>
* This codes was originally wrote by Stijn de Gouw, modified by Evan Yu to adapt to
* our test suite.
* <p>
* https://github.com/abstools/java-timsort-bug
* https://github.com/abstools/java-timsort-bug/blob/master/LICENSE
*/
public class TestTimSort {
private static final int MIN_MERGE = 32;
/**
* Returns an array of integers that demonstrate the bug in TimSort
*/
public static int[] getTimSortBugTestSet(int length) {
int minRun = minRunLength(length);
List<Long> runs = runsJDKWorstCase(minRun, length);
return createArray(runs, length);
}
private static int minRunLength(int n) {
int r = 0; // Becomes 1 if any 1 bits are shifted off
while (n >= MIN_MERGE) {
r |= (n & 1);
n >>= 1;
}
return n + r;
}
private static int[] createArray(List<Long> runs, int length) {
int[] a = new int[length];
Arrays.fill(a, 0);
int endRun = -1;
for (long len : runs) {
a[endRun += len] = 1;
}
a[length - 1] = 0;
return a;
}
/**
* Fills <code>runs</code> with a sequence of run lengths of the form<br>
* Y_n x_{n,1} x_{n,2} ... x_{n,l_n} <br>
* Y_{n-1} x_{n-1,1} x_{n-1,2} ... x_{n-1,l_{n-1}} <br>
* ... <br>
* Y_1 x_{1,1} x_{1,2} ... x_{1,l_1}<br>
* The Y_i's are chosen to satisfy the invariant throughout execution,
* but the x_{i,j}'s are merged (by <code>TimSort.mergeCollapse</code>)
* into an X_i that violates the invariant.
*
* @param length The sum of all run lengths that will be added to <code>runs</code>.
*/
private static List<Long> runsJDKWorstCase(int minRun, int length) {
List<Long> runs = new ArrayList<>();
long runningTotal = 0, Y = minRun + 4, X = minRun;
while (runningTotal + Y + X <= length) {
runningTotal += X + Y;
generateJDKWrongElem(runs, minRun, X);
runs.add(0, Y);
// X_{i+1} = Y_i + x_{i,1} + 1, since runs.get(1) = x_{i,1}
X = Y + runs.get(1) + 1;
// Y_{i+1} = X_{i+1} + Y_i + 1
Y += X + 1;
}
if (runningTotal + X <= length) {
runningTotal += X;
generateJDKWrongElem(runs, minRun, X);
}
runs.add(length - runningTotal);
return runs;
}
/**
* Adds a sequence x_1, ..., x_n of run lengths to <code>runs</code> such that:<br>
* 1. X = x_1 + ... + x_n <br>
* 2. x_j >= minRun for all j <br>
* 3. x_1 + ... + x_{j-2} < x_j < x_1 + ... + x_{j-1} for all j <br>
* These conditions guarantee that TimSort merges all x_j's one by one
* (resulting in X) using only merges on the second-to-last element.
*
* @param X The sum of the sequence that should be added to runs.
*/
private static void generateJDKWrongElem(List<Long> runs, int minRun, long X) {
for (long newTotal; X >= 2 * minRun + 1; X = newTotal) {
//Default strategy
newTotal = X / 2 + 1;
//Specialized strategies
if (3 * minRun + 3 <= X && X <= 4 * minRun + 1) {
// add x_1=MIN+1, x_2=MIN, x_3=X-newTotal to runs
newTotal = 2 * minRun + 1;
} else if (5 * minRun + 5 <= X && X <= 6 * minRun + 5) {
// add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=X-newTotal to runs
newTotal = 3 * minRun + 3;
} else if (8 * minRun + 9 <= X && X <= 10 * minRun + 9) {
// add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=X-newTotal to runs
newTotal = 5 * minRun + 5;
} else if (13 * minRun + 15 <= X && X <= 16 * minRun + 17) {
// add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=3MIN+4, x_6=X-newTotal to runs
newTotal = 8 * minRun + 9;
}
runs.add(0, X - newTotal);
}
runs.add(0, X);
}
}

View File

@@ -1,25 +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 com.actiontech.dble.memory.unsafe.sort;
public class UnsafeExternalSorterRadixSortSuite extends UnsafeExternalSorterSuite {
@Override
protected boolean shouldUseRadixSort() {
return true;
}
}

View File

@@ -1,423 +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 com.actiontech.dble.memory.unsafe.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.TestMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.storage.DataNodeDiskManager;
import com.actiontech.dble.memory.unsafe.storage.DataNodeFileManager;
import com.actiontech.dble.memory.unsafe.storage.SerializerManager;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import com.actiontech.dble.memory.unsafe.utils.sort.*;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.*;
public class UnsafeExternalSorterSuite {
private static final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorterSuite.class);
static final TestMemoryManager memoryManager =
new TestMemoryManager(new ServerPropertyConf().
set("server.memory.offHeap.enabled", "false"));
static final DataNodeMemoryManager DATA_NODE_MEMORY_MANAGER = new DataNodeMemoryManager(memoryManager, 0);
static final SerializerManager serializerManager = new SerializerManager();
static final ServerPropertyConf conf = new ServerPropertyConf();
static final DataNodeDiskManager blockManager = new DataNodeDiskManager(conf, true);
static DataNodeFileManager diskBlockManager;
// Use integer comparison for comparing prefixes (which are partition ids, in this case)
final PrefixComparator prefixComparator = PrefixComparators.LONG;
// Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so
// use a dummy comparator
final RecordComparator recordComparator = new RecordComparator() {
@Override
public int compare(
Object leftBaseObject,
long leftBaseOffset,
Object rightBaseObject,
long rightBaseOffset) {
return 0;
}
};
static File tempDir;
static {
try {
diskBlockManager = blockManager.diskBlockManager();
} catch (IOException e) {
logger.error(e.getMessage());
}
}
protected boolean shouldUseRadixSort() {
return false;
}
private final long pageSizeBytes = new ServerPropertyConf().
getSizeAsBytes("server.buffer.pageSize", "4m");
@Before
public void setUp() {
}
@After
public void tearDown() throws IOException {
try {
assertEquals(0L, DATA_NODE_MEMORY_MANAGER.cleanUpAllAllocatedMemory());
} finally {
//Utils.deleteRecursively(tempDir);
// tempDir = null;
}
}
private void assertSpillFilesWereCleanedUp() {
return;
}
private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception {
final int[] arr = new int[]{value};
sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value);
}
private static void insertRecord(
UnsafeExternalSorter sorter,
int[] record,
long prefix) throws IOException {
sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix);
}
private UnsafeExternalSorter newSorter() throws IOException {
return UnsafeExternalSorter.create(
DATA_NODE_MEMORY_MANAGER,
blockManager,
serializerManager,
recordComparator,
prefixComparator,
/* initialSize */ 1024,
pageSizeBytes,
shouldUseRadixSort(), true);
}
@Test
public void testSortingOnlyByPrefix() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
insertNumber(sorter, 5);
insertNumber(sorter, 1);
insertNumber(sorter, 3);
sorter.spill();
insertNumber(sorter, 4);
sorter.spill();
insertNumber(sorter, 2);
UnsafeSorterIterator iter = sorter.getSortedIterator();
for (int i = 1; i <= 5; i++) {
iter.loadNext();
assertEquals(i, iter.getKeyPrefix());
assertEquals(4, iter.getRecordLength());
assertEquals(i, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void testSortingEmptyArrays() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
sorter.insertRecord(null, 0, 0, 0);
sorter.insertRecord(null, 0, 0, 0);
sorter.spill();
sorter.insertRecord(null, 0, 0, 0);
sorter.spill();
sorter.insertRecord(null, 0, 0, 0);
sorter.insertRecord(null, 0, 0, 0);
UnsafeSorterIterator iter = sorter.getSortedIterator();
for (int i = 1; i <= 5; i++) {
iter.loadNext();
assertEquals(0, iter.getKeyPrefix());
assertEquals(0, iter.getRecordLength());
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void testSortTimeMetric() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
long prevSortTime = sorter.getSortTimeNanos();
assertEquals(prevSortTime, 0);
sorter.insertRecord(null, 0, 0, 0);
sorter.spill();
assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime));
prevSortTime = sorter.getSortTimeNanos();
sorter.spill(); // no sort needed
assertEquals(sorter.getSortTimeNanos(), prevSortTime);
sorter.insertRecord(null, 0, 0, 0);
UnsafeSorterIterator iter = sorter.getSortedIterator();
assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime));
}
@Test
public void spillingOccursInResponseToMemoryPressure() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
// This should be enough records to completely fill up a data page:
final int numRecords = (int) (pageSizeBytes / (4 + 4));
for (int i = 0; i < numRecords; i++) {
insertNumber(sorter, numRecords - i);
}
assertEquals(1, sorter.getNumberOfAllocatedPages());
memoryManager.markExecutionAsOutOfMemoryOnce();
// The insertion of this record should trigger a spill:
insertNumber(sorter, 0);
// Ensure that spill files were created
// assertThat(tempDir.listFiles().length, greaterThanOrEqualTo(1));
// Read back the sorted data:
UnsafeSorterIterator iter = sorter.getSortedIterator();
int i = 0;
while (iter.hasNext()) {
iter.loadNext();
assertEquals(i, iter.getKeyPrefix());
assertEquals(4, iter.getRecordLength());
assertEquals(i, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
i++;
}
assertEquals(numRecords + 1, i);
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void testFillingPage() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
byte[] record = new byte[16];
while (sorter.getNumberOfAllocatedPages() < 2) {
sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0);
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void sortingRecordsThatExceedPageSize() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
final int[] largeRecord = new int[(int) pageSizeBytes + 16];
Arrays.fill(largeRecord, 456);
final int[] smallRecord = new int[100];
Arrays.fill(smallRecord, 123);
insertRecord(sorter, largeRecord, 456);
sorter.spill();
insertRecord(sorter, smallRecord, 123);
sorter.spill();
insertRecord(sorter, smallRecord, 123);
insertRecord(sorter, largeRecord, 456);
UnsafeSorterIterator iter = sorter.getSortedIterator();
// Small record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(123, iter.getKeyPrefix());
assertEquals(smallRecord.length * 4, iter.getRecordLength());
assertEquals(123, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Small record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(123, iter.getKeyPrefix());
assertEquals(smallRecord.length * 4, iter.getRecordLength());
assertEquals(123, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Large record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(456, iter.getKeyPrefix());
assertEquals(largeRecord.length * 4, iter.getRecordLength());
assertEquals(456, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Large record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(456, iter.getKeyPrefix());
assertEquals(largeRecord.length * 4, iter.getRecordLength());
assertEquals(456, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
assertFalse(iter.hasNext());
//sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void forcedSpillingWithReadIterator() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
long[] record = new long[100];
int recordSize = record.length * 8;
int n = (int) pageSizeBytes / recordSize * 3;
for (int i = 0; i < n; i++) {
record[0] = (long) i;
sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
}
assertTrue(sorter.getNumberOfAllocatedPages() >= 2);
UnsafeExternalSorter.SpillableIterator iter =
(UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
int lastv = 0;
for (int i = 0; i < n / 3; i++) {
iter.hasNext();
iter.loadNext();
assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
lastv = i;
}
assertTrue(iter.spill() > 0);
assertEquals(0, iter.spill());
assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv);
for (int i = n / 3; i < n; i++) {
iter.hasNext();
iter.loadNext();
assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void forcedSpillingWithNotReadIterator() throws Exception {
final UnsafeExternalSorter sorter = newSorter();
long[] record = new long[100];
int recordSize = record.length * 8;
int n = (int) pageSizeBytes / recordSize * 3;
for (int i = 0; i < n; i++) {
record[0] = (long) i;
sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
}
assertTrue(sorter.getNumberOfAllocatedPages() >= 2);
UnsafeExternalSorter.SpillableIterator iter =
(UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
assertTrue(iter.spill() > 0);
assertEquals(0, iter.spill());
for (int i = 0; i < n; i++) {
iter.hasNext();
iter.loadNext();
assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void forcedSpillingWithoutComparator() throws Exception {
final UnsafeExternalSorter sorter = UnsafeExternalSorter.create(
DATA_NODE_MEMORY_MANAGER,
blockManager,
serializerManager,
null,
null,
/* initialSize */ 1024,
pageSizeBytes,
shouldUseRadixSort(), true);
long[] record = new long[100];
int recordSize = record.length * 8;
int n = (int) pageSizeBytes / recordSize * 3;
int batch = n / 4;
for (int i = 0; i < n; i++) {
record[0] = (long) i;
sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
if (i % batch == batch - 1) {
sorter.spill();
}
}
UnsafeSorterIterator iter = sorter.getIterator();
for (int i = 0; i < n; i++) {
iter.hasNext();
iter.loadNext();
assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
@Test
public void testPeakMemoryUsed() throws Exception {
final long recordLengthBytes = 8;
final long pageSizeBytes = 256;
final long numRecordsPerPage = pageSizeBytes / recordLengthBytes;
final UnsafeExternalSorter sorter = UnsafeExternalSorter.create(
DATA_NODE_MEMORY_MANAGER,
blockManager,
serializerManager,
recordComparator,
prefixComparator,
1024,
pageSizeBytes,
shouldUseRadixSort(), true);
// Peak memory should be monotonically increasing. More specifically, every time
// we allocate a new page it should increase by exactly the size of the page.
long previousPeakMemory = sorter.getPeakMemoryUsedBytes();
long newPeakMemory;
try {
for (int i = 0; i < numRecordsPerPage * 10; i++) {
insertNumber(sorter, i);
newPeakMemory = sorter.getPeakMemoryUsedBytes();
if (i % numRecordsPerPage == 0) {
// We allocated a new page for this record, so peak memory should change
assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory);
} else {
assertEquals(previousPeakMemory, newPeakMemory);
}
previousPeakMemory = newPeakMemory;
}
// Spilling should not change peak memory
sorter.spill();
newPeakMemory = sorter.getPeakMemoryUsedBytes();
assertEquals(previousPeakMemory, newPeakMemory);
for (int i = 0; i < numRecordsPerPage; i++) {
insertNumber(sorter, i);
}
newPeakMemory = sorter.getPeakMemoryUsedBytes();
assertEquals(previousPeakMemory, newPeakMemory);
} finally {
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
}
}
}

View File

@@ -1,25 +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 com.actiontech.dble.memory.unsafe.sort;
public class UnsafeInMemorySorterRadixSortSuite extends UnsafeInMemorySorterSuite {
@Override
protected boolean shouldUseRadixSort() {
return true;
}
}

View File

@@ -1,174 +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 com.actiontech.dble.memory.unsafe.sort;
import com.actiontech.dble.memory.unsafe.Platform;
import com.actiontech.dble.memory.unsafe.memory.MemoryBlock;
import com.actiontech.dble.memory.unsafe.memory.TestMemoryConsumer;
import com.actiontech.dble.memory.unsafe.memory.TestMemoryManager;
import com.actiontech.dble.memory.unsafe.memory.mm.DataNodeMemoryManager;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import com.actiontech.dble.memory.unsafe.utils.sort.*;
import org.junit.Assert;
import org.junit.Test;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.isIn;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.mockito.Mockito.mock;
public class UnsafeInMemorySorterSuite {
protected boolean shouldUseRadixSort() {
return true;
}
private static String getStringFromDataPage(Object baseObject, long baseOffset, int length) {
final byte[] strBytes = new byte[length];
Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, length);
return new String(strBytes, StandardCharsets.UTF_8);
}
@Test
public void testSortingEmptyInput() {
final DataNodeMemoryManager memoryManager = new DataNodeMemoryManager(
new TestMemoryManager(new ServerPropertyConf().set("server.memory.offHeap.enabled", "false")), 0);
final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager);
final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer,
memoryManager,
mock(RecordComparator.class),
mock(PrefixComparator.class),
100,
shouldUseRadixSort(), true);
final UnsafeSorterIterator iter = sorter.getSortedIterator();
Assert.assertFalse(iter.hasNext());
}
@Test
public void testSortingOnlyByIntegerPrefix() throws Exception {
final String[] dataToSort = new String[]{
"Boba",
"Pearls",
"Tapioca",
"Taho",
"Condensed Milk",
"Jasmine",
"Milk Tea",
"Lychee",
"Mango"
};
final DataNodeMemoryManager memoryManager = new DataNodeMemoryManager(
new TestMemoryManager(new ServerPropertyConf().set("server.memory.offHeap.enabled", "false")), 0);
final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager);
final MemoryBlock dataPage = memoryManager.allocatePage(2048, null);
final Object baseObject = dataPage.getBaseObject();
// Write the records into the data page:
long position = dataPage.getBaseOffset();
for (String str : dataToSort) {
final byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
Platform.putInt(baseObject, position, strBytes.length);
position += 4;
Platform.copyMemory(strBytes, Platform.BYTE_ARRAY_OFFSET, baseObject, position, strBytes.length);
position += strBytes.length;
}
// Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so
// use a dummy comparator
final RecordComparator recordComparator = new RecordComparator() {
@Override
public int compare(
Object leftBaseObject,
long leftBaseOffset,
Object rightBaseObject,
long rightBaseOffset) {
return 0;
}
};
// Compute key prefixes based on the records' partition ids
final HashPartitioner hashPartitioner = new HashPartitioner(4);
// Use integer comparison for comparing prefixes (which are partition ids, in this case)
final PrefixComparator prefixComparator = PrefixComparators.LONG;
UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(
consumer, memoryManager, recordComparator,
prefixComparator, dataToSort.length,
shouldUseRadixSort(), true);
// Given a page of records, insert those records into the sorter one-by-one:
position = dataPage.getBaseOffset();
System.out.println("(0)address = " + position);
for (int i = 0; i < dataToSort.length; i++) {
if (!sorter.hasSpaceForAnotherRecord()) {
sorter.expandPointerArray(consumer.allocateLongArray(sorter.getMemoryUsage() / 8 * 2));
}
// position now points to the start of a record (which holds its length).
final int recordLength = Platform.getInt(baseObject, position);
final long address = memoryManager.encodePageNumberAndOffset(dataPage, position);
final String str = getStringFromDataPage(baseObject, position + 4, recordLength);
final int partitionId = hashPartitioner.getPartition(str);
System.out.println("(" + partitionId + "," + str + ")");
sorter.insertRecord(address, partitionId);
position += 4 + recordLength;
}
final UnsafeSorterIterator iter = sorter.getSortedIterator();
int iterLength = 0;
long prevPrefix = -1;
Arrays.sort(dataToSort);
while (iter.hasNext()) {
iter.loadNext();
final String str = getStringFromDataPage(iter.getBaseObject(), iter.getBaseOffset(), iter.getRecordLength());
final long keyPrefix = iter.getKeyPrefix();
assertThat(str, isIn(Arrays.asList(dataToSort)));
assertThat(keyPrefix, greaterThanOrEqualTo(prevPrefix));
prevPrefix = keyPrefix;
iterLength++;
}
assertEquals(dataToSort.length, iterLength);
}
}

View File

@@ -1,112 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
import com.actiontech.dble.memory.unsafe.utils.ServerPropertyConf;
import com.google.common.io.Closeables;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
/**
* Created by zagnix on 2016/6/4.
*/
public class BlockManagerTest {
private static final Logger logger = LoggerFactory.getLogger(BlockManagerTest.class);
@Test
public void testNewDiskBlockManager() throws IOException {
ServerPropertyConf conf = new ServerPropertyConf();
SerializerManager serializerManager = new SerializerManager();
DataNodeDiskManager blockManager = new DataNodeDiskManager(conf, true);
DataNodeFileManager diskBlockManager = blockManager.diskBlockManager();
/**
* generate a file
*/
File file = diskBlockManager.getFile("file1");
FileOutputStream fos = new FileOutputStream(file);
BufferedOutputStream bos = new BufferedOutputStream(fos);
bos.write("KOKKKKKK".getBytes());
bos.flush();
bos.close();
fos.close();
/**
* read the file
*/
File file1 = diskBlockManager.getFile("file1");
FileInputStream ios = new FileInputStream(file1);
BufferedInputStream bin = new BufferedInputStream(ios);
byte[] str = new byte["KOKKKKKK".getBytes().length];
int size = bin.read(str);
bin.close();
ios.close();
Assert.assertEquals("KOKKKKKK", new String(str));
File file2 = diskBlockManager.getFile("file1");
DiskRowWriter writer = blockManager.
getDiskWriter(file2, DummySerializerInstance.INSTANCE, 1024 * 1024);
byte[] writeBuffer = new byte[4];
int v = 4;
writeBuffer[0] = (byte) (v >>> 24);
writeBuffer[1] = (byte) (v >>> 16);
writeBuffer[2] = (byte) (v >>> 8);
writeBuffer[3] = (byte) (v >>> 0);
writer.write(writeBuffer, 0, 4);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.write("you are ok? 1111111111111".getBytes(), 0, "you are ok? 1111111111111".getBytes().length);
writer.close();
try {
Thread.sleep(100);
} catch (InterruptedException e) {
logger.error(e.getMessage());
}
assert (file2.length() > 0);
final BufferedInputStream bs = new BufferedInputStream(new FileInputStream(file2));
try {
InputStream in = serializerManager.wrapForCompression(null, bs);
DataInputStream din = new DataInputStream(in);
int numRecords = din.readInt();
Assert.assertEquals(4, numRecords);
din.close();
in.close();
bs.close();
} catch (IOException e) {
Closeables.close(bs, /* swallowIOException = */ true);
throw e;
}
}
@Test
public void testNewDiskBlockWriter() {
ServerPropertyConf conf = new ServerPropertyConf();
SerializerManager serializerManager = new SerializerManager();
}
}

View File

@@ -1,41 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.storage;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
/**
* Created by zagnix on 2016/6/4.
*/
public class SerializerManagerTest {
@Test
public void testNewSerializerManager() throws IOException {
SerializerManager serializerManager = new SerializerManager();
final int[] value = new int[1];
OutputStream s = serializerManager.wrapForCompression(null, new OutputStream() {
@Override
public void write(int b) throws IOException {
value[0] = b;
}
});
s.write(10);
Assert.assertEquals(10, value[0]);
InputStream in = serializerManager.wrapForCompression(null, new InputStream() {
@Override
public int read() throws IOException {
return 10;
}
});
Assert.assertEquals(10, in.read());
}
}

View File

@@ -1,239 +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 com.actiontech.dble.memory.unsafe.types;
import org.junit.Test;
import static org.junit.Assert.*;
public class CalendarIntervalSuite {
@Test
public void equalsTest() {
CalendarInterval i1 = new CalendarInterval(3, 123);
CalendarInterval i2 = new CalendarInterval(3, 321);
CalendarInterval i3 = new CalendarInterval(1, 123);
CalendarInterval i4 = new CalendarInterval(3, 123);
assertNotSame(i1, i2);
assertNotSame(i1, i3);
assertNotSame(i2, i3);
assertEquals(i1, i4);
}
@Test
public void toStringTest() {
CalendarInterval i;
i = new CalendarInterval(34, 0);
assertEquals("interval 2 years 10 months", i.toString());
i = new CalendarInterval(-34, 0);
assertEquals("interval -2 years -10 months", i.toString());
i = new CalendarInterval(0, 3 * CalendarInterval.MICROS_PER_WEEK + 13 * CalendarInterval.MICROS_PER_HOUR + 123);
assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString());
i = new CalendarInterval(0, -3 * CalendarInterval.MICROS_PER_WEEK - 13 * CalendarInterval.MICROS_PER_HOUR - 123);
assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString());
i = new CalendarInterval(34, 3 * CalendarInterval.MICROS_PER_WEEK + 13 * CalendarInterval.MICROS_PER_HOUR + 123);
assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString());
}
@Test
public void fromStringTest() {
testSingleUnit("year", 3, 36, 0);
testSingleUnit("month", 3, 3, 0);
testSingleUnit("week", 3, 0, 3 * CalendarInterval.MICROS_PER_WEEK);
testSingleUnit("day", 3, 0, 3 * CalendarInterval.MICROS_PER_DAY);
testSingleUnit("hour", 3, 0, 3 * CalendarInterval.MICROS_PER_HOUR);
testSingleUnit("minute", 3, 0, 3 * CalendarInterval.MICROS_PER_MINUTE);
testSingleUnit("second", 3, 0, 3 * CalendarInterval.MICROS_PER_SECOND);
testSingleUnit("millisecond", 3, 0, 3 * CalendarInterval.MICROS_PER_MILLI);
testSingleUnit("microsecond", 3, 0, 3);
String input;
input = "interval -5 years 23 month";
CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0);
assertEquals(CalendarInterval.fromString(input), result);
input = "interval -5 years 23 month ";
assertEquals(CalendarInterval.fromString(input), result);
input = " interval -5 years 23 month ";
assertEquals(CalendarInterval.fromString(input), result);
// Error cases
input = "interval 3month 1 hour";
assertNull(CalendarInterval.fromString(input));
input = "interval 3 moth 1 hour";
assertNull(CalendarInterval.fromString(input));
input = "interval";
assertNull(CalendarInterval.fromString(input));
input = "int";
assertNull(CalendarInterval.fromString(input));
input = "";
assertNull(CalendarInterval.fromString(input));
input = null;
assertNull(CalendarInterval.fromString(input));
}
@Test
public void fromYearMonthStringTest() {
String input;
CalendarInterval i;
input = "99-10";
i = new CalendarInterval(99 * 12 + 10, 0L);
assertEquals(CalendarInterval.fromYearMonthString(input), i);
input = "-8-10";
i = new CalendarInterval(-8 * 12 - 10, 0L);
assertEquals(CalendarInterval.fromYearMonthString(input), i);
try {
input = "99-15";
CalendarInterval.fromYearMonthString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("month 15 outside range"));
}
}
@Test
public void fromDayTimeStringTest() {
String input;
CalendarInterval i;
input = "5 12:40:30.999999999";
i = new CalendarInterval(0, 5 * CalendarInterval.MICROS_PER_DAY + 12 * CalendarInterval.MICROS_PER_HOUR +
40 * CalendarInterval.MICROS_PER_MINUTE + 30 * CalendarInterval.MICROS_PER_SECOND + 999999L);
assertEquals(CalendarInterval.fromDayTimeString(input), i);
input = "10 0:12:0.888";
i = new CalendarInterval(0, 10 * CalendarInterval.MICROS_PER_DAY + 12 * CalendarInterval.MICROS_PER_MINUTE);
assertEquals(CalendarInterval.fromDayTimeString(input), i);
input = "-3 0:0:0";
i = new CalendarInterval(0, -3 * CalendarInterval.MICROS_PER_DAY);
assertEquals(CalendarInterval.fromDayTimeString(input), i);
try {
input = "5 30:12:20";
CalendarInterval.fromDayTimeString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("hour 30 outside range"));
}
try {
input = "5 30-12";
CalendarInterval.fromDayTimeString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("not match day-time format"));
}
}
@Test
public void fromSingleUnitStringTest() {
String input;
CalendarInterval i;
input = "12";
i = new CalendarInterval(12 * 12, 0L);
assertEquals(CalendarInterval.fromSingleUnitString("year", input), i);
input = "100";
i = new CalendarInterval(0, 100 * CalendarInterval.MICROS_PER_DAY);
assertEquals(CalendarInterval.fromSingleUnitString("day", input), i);
input = "1999.38888";
i = new CalendarInterval(0, 1999 * CalendarInterval.MICROS_PER_SECOND + 38);
assertEquals(CalendarInterval.fromSingleUnitString("second", input), i);
try {
input = String.valueOf(Integer.MAX_VALUE);
CalendarInterval.fromSingleUnitString("year", input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("outside range"));
}
try {
input = String.valueOf(Long.MAX_VALUE / CalendarInterval.MICROS_PER_HOUR + 1);
CalendarInterval.fromSingleUnitString("hour", input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("outside range"));
}
}
@Test
public void addTest() {
String input = "interval 3 month 1 hour";
String input2 = "interval 2 month 100 hour";
CalendarInterval interval = CalendarInterval.fromString(input);
CalendarInterval interval2 = CalendarInterval.fromString(input2);
assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * CalendarInterval.MICROS_PER_HOUR));
input = "interval -10 month -81 hour";
input2 = "interval 75 month 200 hour";
interval = CalendarInterval.fromString(input);
interval2 = CalendarInterval.fromString(input2);
assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * CalendarInterval.MICROS_PER_HOUR));
}
@Test
public void subtractTest() {
String input = "interval 3 month 1 hour";
String input2 = "interval 2 month 100 hour";
CalendarInterval interval = CalendarInterval.fromString(input);
CalendarInterval interval2 = CalendarInterval.fromString(input2);
assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * CalendarInterval.MICROS_PER_HOUR));
input = "interval -10 month -81 hour";
input2 = "interval 75 month 200 hour";
interval = CalendarInterval.fromString(input);
interval2 = CalendarInterval.fromString(input2);
assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * CalendarInterval.MICROS_PER_HOUR));
}
private static void testSingleUnit(String unit, int number, int months, long microseconds) {
String input1 = "interval " + number + " " + unit;
String input2 = "interval " + number + " " + unit + "s";
CalendarInterval result = new CalendarInterval(months, microseconds);
assertEquals(CalendarInterval.fromString(input1), result);
assertEquals(CalendarInterval.fromString(input2), result);
}
}

View File

@@ -1,494 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.memory.unsafe.types;
/*
* 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.
*/
import com.google.common.collect.ImmutableMap;
import org.junit.Test;
import java.io.UnsupportedEncodingException;
import java.util.Arrays;
import java.util.HashMap;
import static org.junit.Assert.*;
public class UTF8StringSuite {
private static void checkBasic(String str, int len) throws UnsupportedEncodingException {
UTF8String s1 = UTF8String.fromString(str);
UTF8String s2 = UTF8String.fromBytes(str.getBytes("utf8"));
assertEquals(s1.numChars(), len);
assertEquals(s2.numChars(), len);
assertEquals(s1.toString(), str);
assertEquals(s2.toString(), str);
assertEquals(s1, s2);
assertEquals(s1.hashCode(), s2.hashCode());
assertEquals(0, s1.compareTo(s2));
assertTrue(s1.contains(s2));
assertTrue(s2.contains(s1));
assertTrue(s1.startsWith(s1.clone()));
assertTrue(s1.endsWith(s1.clone()));
}
@Test
public void basicTest() throws UnsupportedEncodingException {
checkBasic("", 0);
checkBasic("hello", 5);
checkBasic("大 千 世 界", 7);
}
@Test
public void emptyStringTest() {
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString(""));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromBytes(new byte[0]));
assertEquals(0, UTF8String.EMPTY_UTF8.numChars());
assertEquals(0, UTF8String.EMPTY_UTF8.numBytes());
}
@Test
public void prefix() {
assertTrue(UTF8String.fromString("a").getPrefix() - UTF8String.fromString("b").getPrefix() < 0);
assertTrue(UTF8String.fromString("ab").getPrefix() - UTF8String.fromString("b").getPrefix() < 0);
assertTrue(
UTF8String.fromString("abbbbbbbbbbbasdf").getPrefix() - UTF8String.fromString("bbbbbbbbbbbbasdf").getPrefix() < 0);
assertTrue(UTF8String.fromString("").getPrefix() - UTF8String.fromString("a").getPrefix() < 0);
assertTrue(UTF8String.fromString("你好").getPrefix() - UTF8String.fromString("世界").getPrefix() > 0);
byte[] buf1 = {1, 2, 3, 4, 5, 6, 7, 8, 9};
byte[] buf2 = {1, 2, 3};
UTF8String str1 = UTF8String.fromBytes(buf1, 0, 3);
UTF8String str2 = UTF8String.fromBytes(buf1, 0, 8);
UTF8String str3 = UTF8String.fromBytes(buf2);
assertTrue(str1.getPrefix() - str2.getPrefix() < 0);
assertEquals(str1.getPrefix(), str3.getPrefix());
}
@Test
public void compareTo() {
assertTrue(UTF8String.fromString("").compareTo(UTF8String.fromString("a")) < 0);
assertTrue(UTF8String.fromString("abc").compareTo(UTF8String.fromString("ABC")) > 0);
assertTrue(UTF8String.fromString("abc0").compareTo(UTF8String.fromString("abc")) > 0);
assertTrue(UTF8String.fromString("abcabcabc").compareTo(UTF8String.fromString("abcabcabc")) == 0);
assertTrue(UTF8String.fromString("aBcabcabc").compareTo(UTF8String.fromString("Abcabcabc")) > 0);
assertTrue(UTF8String.fromString("Abcabcabc").compareTo(UTF8String.fromString("abcabcabC")) < 0);
assertTrue(UTF8String.fromString("abcabcabc").compareTo(UTF8String.fromString("abcabcabC")) > 0);
assertTrue(UTF8String.fromString("abc").compareTo(UTF8String.fromString("世界")) < 0);
assertTrue(UTF8String.fromString("你好").compareTo(UTF8String.fromString("世界")) > 0);
assertTrue(UTF8String.fromString("你好123").compareTo(UTF8String.fromString("你好122")) > 0);
}
protected static void testUpperandLower(String upper, String lower) {
UTF8String us = UTF8String.fromString(upper);
UTF8String ls = UTF8String.fromString(lower);
assertEquals(ls, us.toLowerCase());
assertEquals(us, ls.toUpperCase());
assertEquals(us, us.toUpperCase());
assertEquals(ls, ls.toLowerCase());
}
@Test
public void upperAndLower() {
testUpperandLower("", "");
testUpperandLower("0123456", "0123456");
testUpperandLower("ABCXYZ", "abcxyz");
testUpperandLower("ЀЁЂѺΏỀ", "ѐёђѻώề");
testUpperandLower("大千世界 数据砖头", "大千世界 数据砖头");
}
@Test
public void titleCase() {
assertEquals(UTF8String.fromString(""), UTF8String.fromString("").toTitleCase());
assertEquals(UTF8String.fromString("Ab Bc Cd"), UTF8String.fromString("ab bc cd").toTitleCase());
assertEquals(UTF8String.fromString("Ѐ Ё Ђ Ѻ Ώ Ề"), UTF8String.fromString("ѐ ё ђ ѻ ώ ề").toTitleCase());
assertEquals(UTF8String.fromString("大千世界 数据砖头"), UTF8String.fromString("大千世界 数据砖头").toTitleCase());
}
@Test
public void concatTest() {
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.concat());
assertNull(UTF8String.concat((UTF8String) null));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.concat(UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.fromString("ab"), UTF8String.concat(UTF8String.fromString("ab")));
assertEquals(UTF8String.fromString("ab"), UTF8String.concat(UTF8String.fromString("a"), UTF8String.fromString("b")));
assertEquals(UTF8String.fromString("abc"), UTF8String.concat(UTF8String.fromString("a"), UTF8String.fromString("b"), UTF8String.fromString("c")));
assertNull(UTF8String.concat(UTF8String.fromString("a"), null, UTF8String.fromString("c")));
assertNull(UTF8String.concat(UTF8String.fromString("a"), null, null));
assertNull(UTF8String.concat(null, null, null));
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.concat(UTF8String.fromString("数据"), UTF8String.fromString("砖头")));
}
@Test
public void concatWsTest() {
// Returns null if the separator is null
assertNull(UTF8String.concatWs(null, (UTF8String) null));
assertNull(UTF8String.concatWs(null, UTF8String.fromString("a")));
// If separator is null, concatWs should skip all null inputs and never return null.
UTF8String sep = UTF8String.fromString("哈哈");
assertEquals(
UTF8String.EMPTY_UTF8,
UTF8String.concatWs(sep, UTF8String.EMPTY_UTF8));
assertEquals(
UTF8String.fromString("ab"),
UTF8String.concatWs(sep, UTF8String.fromString("ab")));
assertEquals(
UTF8String.fromString("a哈哈b"),
UTF8String.concatWs(sep, UTF8String.fromString("a"), UTF8String.fromString("b")));
assertEquals(
UTF8String.fromString("a哈哈b哈哈c"),
UTF8String.concatWs(sep, UTF8String.fromString("a"), UTF8String.fromString("b"), UTF8String.fromString("c")));
assertEquals(
UTF8String.fromString("a哈哈c"),
UTF8String.concatWs(sep, UTF8String.fromString("a"), null, UTF8String.fromString("c")));
assertEquals(
UTF8String.fromString("a"),
UTF8String.concatWs(sep, UTF8String.fromString("a"), null, null));
assertEquals(
UTF8String.EMPTY_UTF8,
UTF8String.concatWs(sep, null, null, null));
assertEquals(
UTF8String.fromString("数据哈哈砖头"),
UTF8String.concatWs(sep, UTF8String.fromString("数据"), UTF8String.fromString("砖头")));
}
@Test
public void contains() {
assertTrue(UTF8String.fromString("hello").contains(UTF8String.fromString("ello")));
assertFalse(UTF8String.fromString("hello").contains(UTF8String.fromString("vello")));
assertFalse(UTF8String.fromString("hello").contains(UTF8String.fromString("hellooo")));
assertTrue(UTF8String.fromString("大千世界").contains(UTF8String.fromString("千世界")));
assertFalse(UTF8String.fromString("大千世界").contains(UTF8String.fromString("世千")));
assertFalse(UTF8String.fromString("大千世界").contains(UTF8String.fromString("大千世界好")));
}
@Test
public void startsWith() {
assertTrue(UTF8String.fromString("hello").startsWith(UTF8String.fromString("hell")));
assertFalse(UTF8String.fromString("hello").startsWith(UTF8String.fromString("ell")));
assertFalse(UTF8String.fromString("hello").startsWith(UTF8String.fromString("hellooo")));
assertTrue(UTF8String.fromString("数据砖头").startsWith(UTF8String.fromString("数据")));
assertFalse(UTF8String.fromString("大千世界").startsWith(UTF8String.fromString("")));
assertFalse(UTF8String.fromString("大千世界").startsWith(UTF8String.fromString("大千世界好")));
}
@Test
public void endsWith() {
assertTrue(UTF8String.fromString("hello").endsWith(UTF8String.fromString("ello")));
assertFalse(UTF8String.fromString("hello").endsWith(UTF8String.fromString("ellov")));
assertFalse(UTF8String.fromString("hello").endsWith(UTF8String.fromString("hhhello")));
assertTrue(UTF8String.fromString("大千世界").endsWith(UTF8String.fromString("世界")));
assertFalse(UTF8String.fromString("大千世界").endsWith(UTF8String.fromString("")));
assertFalse(UTF8String.fromString("数据砖头").endsWith(UTF8String.fromString("我的数据砖头")));
}
@Test
public void substring() {
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("hello").substring(0, 0));
assertEquals(UTF8String.fromString("el"), UTF8String.fromString("hello").substring(1, 3));
assertEquals(UTF8String.fromString(""), UTF8String.fromString("数据砖头").substring(0, 1));
assertEquals(UTF8String.fromString("据砖"), UTF8String.fromString("数据砖头").substring(1, 3));
assertEquals(UTF8String.fromString(""), UTF8String.fromString("数据砖头").substring(3, 5));
assertEquals(UTF8String.fromString("ߵ梷"), UTF8String.fromString("ߵ梷").substring(0, 2));
}
@Test
public void trims() {
assertEquals(UTF8String.fromString("hello"), UTF8String.fromString(" hello ").trim());
assertEquals(UTF8String.fromString("hello "), UTF8String.fromString(" hello ").trimLeft());
assertEquals(UTF8String.fromString(" hello"), UTF8String.fromString(" hello ").trimRight());
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString(" ").trim());
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString(" ").trimLeft());
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString(" ").trimRight());
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString(" 数据砖头 ").trim());
assertEquals(UTF8String.fromString("数据砖头 "), UTF8String.fromString(" 数据砖头 ").trimLeft());
assertEquals(UTF8String.fromString(" 数据砖头"), UTF8String.fromString(" 数据砖头 ").trimRight());
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString("数据砖头").trim());
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString("数据砖头").trimLeft());
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString("数据砖头").trimRight());
}
@Test
public void indexOf() {
assertEquals(0, UTF8String.EMPTY_UTF8.indexOf(UTF8String.EMPTY_UTF8, 0));
assertEquals(-1, UTF8String.EMPTY_UTF8.indexOf(UTF8String.fromString("l"), 0));
assertEquals(0, UTF8String.fromString("hello").indexOf(UTF8String.EMPTY_UTF8, 0));
assertEquals(2, UTF8String.fromString("hello").indexOf(UTF8String.fromString("l"), 0));
assertEquals(3, UTF8String.fromString("hello").indexOf(UTF8String.fromString("l"), 3));
assertEquals(-1, UTF8String.fromString("hello").indexOf(UTF8String.fromString("a"), 0));
assertEquals(2, UTF8String.fromString("hello").indexOf(UTF8String.fromString("ll"), 0));
assertEquals(-1, UTF8String.fromString("hello").indexOf(UTF8String.fromString("ll"), 4));
assertEquals(1, UTF8String.fromString("数据砖头").indexOf(UTF8String.fromString("据砖"), 0));
assertEquals(-1, UTF8String.fromString("数据砖头").indexOf(UTF8String.fromString(""), 3));
assertEquals(0, UTF8String.fromString("数据砖头").indexOf(UTF8String.fromString(""), 0));
assertEquals(3, UTF8String.fromString("数据砖头").indexOf(UTF8String.fromString(""), 0));
}
@Test
public void substring_index() {
assertEquals(UTF8String.fromString("www.apache.org"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), 3));
assertEquals(UTF8String.fromString("www.apache"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), 2));
assertEquals(UTF8String.fromString("www"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), 1));
assertEquals(UTF8String.fromString(""),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), 0));
assertEquals(UTF8String.fromString("org"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), -1));
assertEquals(UTF8String.fromString("apache.org"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), -2));
assertEquals(UTF8String.fromString("www.apache.org"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("."), -3));
// str is empty string
assertEquals(UTF8String.fromString(""),
UTF8String.fromString("").subStringIndex(UTF8String.fromString("."), 1));
// empty string delim
assertEquals(UTF8String.fromString(""),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString(""), 1));
// delim does not exist in str
assertEquals(UTF8String.fromString("www.apache.org"),
UTF8String.fromString("www.apache.org").subStringIndex(UTF8String.fromString("#"), 2));
// delim is 2 chars
assertEquals(UTF8String.fromString("www||apache"),
UTF8String.fromString("www||apache||org").subStringIndex(UTF8String.fromString("||"), 2));
assertEquals(UTF8String.fromString("apache||org"),
UTF8String.fromString("www||apache||org").subStringIndex(UTF8String.fromString("||"), -2));
// non ascii chars
assertEquals(UTF8String.fromString("大千世界大"),
UTF8String.fromString("大千世界大千世界").subStringIndex(UTF8String.fromString(""), 2));
// overlapped delim
assertEquals(UTF8String.fromString("||"), UTF8String.fromString("||||||").subStringIndex(UTF8String.fromString("|||"), 3));
assertEquals(UTF8String.fromString("|||"), UTF8String.fromString("||||||").subStringIndex(UTF8String.fromString("|||"), -4));
}
@Test
public void reverse() {
assertEquals(UTF8String.fromString("olleh"), UTF8String.fromString("hello").reverse());
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.EMPTY_UTF8.reverse());
assertEquals(UTF8String.fromString("者行孙"), UTF8String.fromString("孙行者").reverse());
assertEquals(UTF8String.fromString("者行孙 olleh"), UTF8String.fromString("hello 孙行者").reverse());
}
@Test
public void repeat() {
assertEquals(UTF8String.fromString("数d数d数d数d数d"), UTF8String.fromString("数d").repeat(5));
assertEquals(UTF8String.fromString("数d"), UTF8String.fromString("数d").repeat(1));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("数d").repeat(-1));
}
@Test
public void pad() {
assertEquals(UTF8String.fromString("hel"), UTF8String.fromString("hello").lPad(3, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("hello"), UTF8String.fromString("hello").lPad(5, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("?hello"), UTF8String.fromString("hello").lPad(6, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("???????hello"), UTF8String.fromString("hello").lPad(12, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("?????hello"), UTF8String.fromString("hello").lPad(10, UTF8String.fromString("?????")));
assertEquals(UTF8String.fromString("???????"), UTF8String.EMPTY_UTF8.lPad(7, UTF8String.fromString("?????")));
assertEquals(UTF8String.fromString("hel"), UTF8String.fromString("hello").rPad(3, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("hello"), UTF8String.fromString("hello").rPad(5, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("hello?"), UTF8String.fromString("hello").rPad(6, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("hello???????"), UTF8String.fromString("hello").rPad(12, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("hello?????"), UTF8String.fromString("hello").rPad(10, UTF8String.fromString("?????")));
assertEquals(UTF8String.fromString("???????"), UTF8String.EMPTY_UTF8.rPad(7, UTF8String.fromString("?????")));
assertEquals(UTF8String.fromString("数据砖"), UTF8String.fromString("数据砖头").lPad(3, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("?数据砖头"), UTF8String.fromString("数据砖头").lPad(5, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("??数据砖头"), UTF8String.fromString("数据砖头").lPad(6, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("孙行数据砖头"), UTF8String.fromString("数据砖头").lPad(6, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.fromString("孙行者数据砖头"), UTF8String.fromString("数据砖头").lPad(7, UTF8String.fromString("孙行者")));
assertEquals(
UTF8String.fromString("孙行者孙行者孙行数据砖头"),
UTF8String.fromString("数据砖头").lPad(12, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.fromString("数据砖"), UTF8String.fromString("数据砖头").rPad(3, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("数据砖头?"), UTF8String.fromString("数据砖头").rPad(5, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("数据砖头??"), UTF8String.fromString("数据砖头").rPad(6, UTF8String.fromString("????")));
assertEquals(UTF8String.fromString("数据砖头孙行"), UTF8String.fromString("数据砖头").rPad(6, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.fromString("数据砖头孙行者"), UTF8String.fromString("数据砖头").rPad(7, UTF8String.fromString("孙行者")));
assertEquals(
UTF8String.fromString("数据砖头孙行者孙行者孙行"),
UTF8String.fromString("数据砖头").rPad(12, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("数据砖头").lPad(-10, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("数据砖头").lPad(-10, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString("数据砖头").lPad(5, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.fromString("数据砖"), UTF8String.fromString("数据砖头").lPad(3, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.EMPTY_UTF8.lPad(3, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("数据砖头").rPad(-10, UTF8String.fromString("孙行者")));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.fromString("数据砖头").rPad(-10, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.fromString("数据砖头"), UTF8String.fromString("数据砖头").rPad(5, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.fromString("数据砖"), UTF8String.fromString("数据砖头").rPad(3, UTF8String.EMPTY_UTF8));
assertEquals(UTF8String.EMPTY_UTF8, UTF8String.EMPTY_UTF8.rPad(3, UTF8String.EMPTY_UTF8));
}
@Test
public void substringSQL() {
UTF8String e = UTF8String.fromString("example");
assertEquals(e.substringSQL(0, 2), UTF8String.fromString("ex"));
assertEquals(e.substringSQL(1, 2), UTF8String.fromString("ex"));
assertEquals(e.substringSQL(0, 7), UTF8String.fromString("example"));
assertEquals(e.substringSQL(1, 2), UTF8String.fromString("ex"));
assertEquals(e.substringSQL(0, 100), UTF8String.fromString("example"));
assertEquals(e.substringSQL(1, 100), UTF8String.fromString("example"));
assertEquals(e.substringSQL(2, 2), UTF8String.fromString("xa"));
assertEquals(e.substringSQL(1, 6), UTF8String.fromString("exampl"));
assertEquals(e.substringSQL(2, 100), UTF8String.fromString("xample"));
assertEquals(e.substringSQL(0, 0), UTF8String.fromString(""));
assertEquals(e.substringSQL(100, 4), UTF8String.EMPTY_UTF8);
assertEquals(e.substringSQL(0, Integer.MAX_VALUE), UTF8String.fromString("example"));
assertEquals(e.substringSQL(1, Integer.MAX_VALUE), UTF8String.fromString("example"));
assertEquals(e.substringSQL(2, Integer.MAX_VALUE), UTF8String.fromString("xample"));
}
@Test
public void split() {
assertTrue(Arrays.equals(UTF8String.fromString("ab,def,ghi").split(UTF8String.fromString(","), -1),
new UTF8String[]{UTF8String.fromString("ab"), UTF8String.fromString("def"), UTF8String.fromString("ghi")}));
assertTrue(Arrays.equals(UTF8String.fromString("ab,def,ghi").split(UTF8String.fromString(","), 2),
new UTF8String[]{UTF8String.fromString("ab"), UTF8String.fromString("def,ghi")}));
assertTrue(Arrays.equals(UTF8String.fromString("ab,def,ghi").split(UTF8String.fromString(","), 2),
new UTF8String[]{UTF8String.fromString("ab"), UTF8String.fromString("def,ghi")}));
}
@Test
public void levenshteinDistance() {
assertEquals(0, UTF8String.EMPTY_UTF8.levenshteinDistance(UTF8String.EMPTY_UTF8));
assertEquals(1, UTF8String.EMPTY_UTF8.levenshteinDistance(UTF8String.fromString("a")));
assertEquals(7, UTF8String.fromString("aaapppp").levenshteinDistance(UTF8String.EMPTY_UTF8));
assertEquals(1, UTF8String.fromString("frog").levenshteinDistance(UTF8String.fromString("fog")));
assertEquals(3, UTF8String.fromString("fly").levenshteinDistance(UTF8String.fromString("ant")));
assertEquals(7, UTF8String.fromString("elephant").levenshteinDistance(UTF8String.fromString("hippo")));
assertEquals(7, UTF8String.fromString("hippo").levenshteinDistance(UTF8String.fromString("elephant")));
assertEquals(8, UTF8String.fromString("hippo").levenshteinDistance(UTF8String.fromString("zzzzzzzz")));
assertEquals(1, UTF8String.fromString("hello").levenshteinDistance(UTF8String.fromString("hallo")));
assertEquals(4, UTF8String.fromString("世界千世").levenshteinDistance(UTF8String.fromString("千a世b")));
}
@Test
public void translate() {
assertEquals(
UTF8String.fromString("1a2s3ae"),
UTF8String.fromString("translate").translate(ImmutableMap.of(
'r', '1',
'n', '2',
'l', '3',
't', '\0'
)));
assertEquals(
UTF8String.fromString("translate"),
UTF8String.fromString("translate").translate(new HashMap<Character, Character>()));
assertEquals(
UTF8String.fromString("asae"),
UTF8String.fromString("translate").translate(ImmutableMap.of(
'r', '\0',
'n', '\0',
'l', '\0',
't', '\0'
)));
assertEquals(
UTF8String.fromString("aa世b"),
UTF8String.fromString("花花世界").translate(ImmutableMap.of(
'花', 'a',
'界', 'b'
)));
}
@Test
public void createBlankString() {
assertEquals(UTF8String.fromString(" "), UTF8String.blankString(1));
assertEquals(UTF8String.fromString(" "), UTF8String.blankString(2));
assertEquals(UTF8String.fromString(" "), UTF8String.blankString(3));
assertEquals(UTF8String.fromString(""), UTF8String.blankString(0));
}
@Test
public void findInSet() {
assertEquals(1, UTF8String.fromString("ab").findInSet(UTF8String.fromString("ab")));
assertEquals(2, UTF8String.fromString("a,b").findInSet(UTF8String.fromString("b")));
assertEquals(3, UTF8String.fromString("abc,b,ab,c,def").findInSet(UTF8String.fromString("ab")));
assertEquals(1, UTF8String.fromString("ab,abc,b,ab,c,def").findInSet(UTF8String.fromString("ab")));
assertEquals(4, UTF8String.fromString(",,,ab,abc,b,ab,c,def").findInSet(UTF8String.fromString("ab")));
assertEquals(1, UTF8String.fromString(",ab,abc,b,ab,c,def").findInSet(UTF8String.fromString("")));
assertEquals(4, UTF8String.fromString("数据砖头,abc,b,ab,c,def").findInSet(UTF8String.fromString("ab")));
assertEquals(6, UTF8String.fromString("数据砖头,abc,b,ab,c,def").findInSet(UTF8String.fromString("def")));
}
@Test
public void soundex() {
assertEquals(UTF8String.fromString("Robert").soundex(), UTF8String.fromString("R163"));
assertEquals(UTF8String.fromString("Rupert").soundex(), UTF8String.fromString("R163"));
assertEquals(UTF8String.fromString("Rubin").soundex(), UTF8String.fromString("R150"));
assertEquals(UTF8String.fromString("Ashcraft").soundex(), UTF8String.fromString("A261"));
assertEquals(UTF8String.fromString("Ashcroft").soundex(), UTF8String.fromString("A261"));
assertEquals(UTF8String.fromString("Burroughs").soundex(), UTF8String.fromString("B620"));
assertEquals(UTF8String.fromString("Burrows").soundex(), UTF8String.fromString("B620"));
assertEquals(UTF8String.fromString("Ekzampul").soundex(), UTF8String.fromString("E251"));
assertEquals(UTF8String.fromString("Example").soundex(), UTF8String.fromString("E251"));
assertEquals(UTF8String.fromString("Ellery").soundex(), UTF8String.fromString("E460"));
assertEquals(UTF8String.fromString("Euler").soundex(), UTF8String.fromString("E460"));
assertEquals(UTF8String.fromString("Ghosh").soundex(), UTF8String.fromString("G200"));
assertEquals(UTF8String.fromString("Gauss").soundex(), UTF8String.fromString("G200"));
assertEquals(UTF8String.fromString("Gutierrez").soundex(), UTF8String.fromString("G362"));
assertEquals(UTF8String.fromString("Heilbronn").soundex(), UTF8String.fromString("H416"));
assertEquals(UTF8String.fromString("Hilbert").soundex(), UTF8String.fromString("H416"));
assertEquals(UTF8String.fromString("Jackson").soundex(), UTF8String.fromString("J250"));
assertEquals(UTF8String.fromString("Kant").soundex(), UTF8String.fromString("K530"));
assertEquals(UTF8String.fromString("Knuth").soundex(), UTF8String.fromString("K530"));
assertEquals(UTF8String.fromString("Lee").soundex(), UTF8String.fromString("L000"));
assertEquals(UTF8String.fromString("Lukasiewicz").soundex(), UTF8String.fromString("L222"));
assertEquals(UTF8String.fromString("Lissajous").soundex(), UTF8String.fromString("L222"));
assertEquals(UTF8String.fromString("Ladd").soundex(), UTF8String.fromString("L300"));
assertEquals(UTF8String.fromString("Lloyd").soundex(), UTF8String.fromString("L300"));
assertEquals(UTF8String.fromString("Moses").soundex(), UTF8String.fromString("M220"));
assertEquals(UTF8String.fromString("O'Hara").soundex(), UTF8String.fromString("O600"));
assertEquals(UTF8String.fromString("Pfister").soundex(), UTF8String.fromString("P236"));
assertEquals(UTF8String.fromString("Rubin").soundex(), UTF8String.fromString("R150"));
assertEquals(UTF8String.fromString("Robert").soundex(), UTF8String.fromString("R163"));
assertEquals(UTF8String.fromString("Rupert").soundex(), UTF8String.fromString("R163"));
assertEquals(UTF8String.fromString("Soundex").soundex(), UTF8String.fromString("S532"));
assertEquals(UTF8String.fromString("Sownteks").soundex(), UTF8String.fromString("S532"));
assertEquals(UTF8String.fromString("Tymczak").soundex(), UTF8String.fromString("T522"));
assertEquals(UTF8String.fromString("VanDeusen").soundex(), UTF8String.fromString("V532"));
assertEquals(UTF8String.fromString("Washington").soundex(), UTF8String.fromString("W252"));
assertEquals(UTF8String.fromString("Wheaton").soundex(), UTF8String.fromString("W350"));
assertEquals(UTF8String.fromString("a").soundex(), UTF8String.fromString("A000"));
assertEquals(UTF8String.fromString("ab").soundex(), UTF8String.fromString("A100"));
assertEquals(UTF8String.fromString("abc").soundex(), UTF8String.fromString("A120"));
assertEquals(UTF8String.fromString("abcd").soundex(), UTF8String.fromString("A123"));
assertEquals(UTF8String.fromString("").soundex(), UTF8String.fromString(""));
assertEquals(UTF8String.fromString("123").soundex(), UTF8String.fromString("123"));
assertEquals(UTF8String.fromString("世界千世").soundex(), UTF8String.fromString("世界千世"));
}
}

View File

@@ -1,19 +0,0 @@
/*
* Copyright (C) 2016-2017 ActionTech.
* based on code by MyCATCopyrightHolder Copyright (c) 2013, OpenCloudDB/MyCAT.
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
*/
package com.actiontech.dble.postgres;
import org.junit.Test;
/**
* @author mycat
*/
public class PostgresTest {
@Test
public void testNothing() {
}
}