mirror of
https://github.com/actiontech/dble.git
synced 2026-01-05 12:20:19 -06:00
delete_merge_off_heap
This commit is contained in:
@@ -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"/>
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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 >= 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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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 << 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 << 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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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 << 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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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 doesn’t flush the underlying stream we explicitly flush both the
|
||||
// serializer stream and the lower level stream.
|
||||
objOut.flush();
|
||||
bs.flush();
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Reverts writes that haven’t 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 writer’s 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();
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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');
|
||||
}
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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() {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
@@ -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]));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
*
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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 {
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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));
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
@@ -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)));
|
||||
// }
|
||||
|
||||
};
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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("世界千世"));
|
||||
}
|
||||
}
|
||||
@@ -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() {
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user