HDDS-850. ReadStateMachineData hits OverlappingFileLockException in ContainerStateMachine. Contributed by Shashikant Banerjee.
This commit is contained in:
parent
7eb0d3a324
commit
5e102f9aa5
@ -93,6 +93,14 @@ public final class ScmConfigKeys {
|
|||||||
public static final String DFS_CONTAINER_RATIS_LOG_QUEUE_SIZE =
|
public static final String DFS_CONTAINER_RATIS_LOG_QUEUE_SIZE =
|
||||||
"dfs.container.ratis.log.queue.size";
|
"dfs.container.ratis.log.queue.size";
|
||||||
public static final int DFS_CONTAINER_RATIS_LOG_QUEUE_SIZE_DEFAULT = 128;
|
public static final int DFS_CONTAINER_RATIS_LOG_QUEUE_SIZE_DEFAULT = 128;
|
||||||
|
|
||||||
|
// expiry interval stateMachineData cache entry inside containerStateMachine
|
||||||
|
public static final String
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL =
|
||||||
|
"dfs.container.ratis.statemachine.cache.expiry.interval";
|
||||||
|
public static final String
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT =
|
||||||
|
"10s";
|
||||||
public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY =
|
public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY =
|
||||||
"dfs.ratis.client.request.timeout.duration";
|
"dfs.ratis.client.request.timeout.duration";
|
||||||
public static final TimeDuration
|
public static final TimeDuration
|
||||||
|
@ -249,6 +249,15 @@ public final class OzoneConfigKeys {
|
|||||||
DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
|
||||||
ScmConfigKeys.DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT;
|
ScmConfigKeys.DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT;
|
||||||
|
|
||||||
|
public static final String
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL =
|
||||||
|
ScmConfigKeys.
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL;
|
||||||
|
public static final String
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT =
|
||||||
|
ScmConfigKeys.
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT;
|
||||||
|
|
||||||
public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
|
public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
|
||||||
"dfs.container.ratis.datanode.storage.dir";
|
"dfs.container.ratis.datanode.storage.dir";
|
||||||
public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY =
|
public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY =
|
||||||
|
@ -392,6 +392,7 @@ message WriteChunkResponseProto {
|
|||||||
message ReadChunkRequestProto {
|
message ReadChunkRequestProto {
|
||||||
required DatanodeBlockID blockID = 1;
|
required DatanodeBlockID blockID = 1;
|
||||||
required ChunkInfo chunkData = 2;
|
required ChunkInfo chunkData = 2;
|
||||||
|
optional bool readFromTmpFile = 3 [default = false];
|
||||||
}
|
}
|
||||||
|
|
||||||
message ReadChunkResponseProto {
|
message ReadChunkResponseProto {
|
||||||
|
@ -189,6 +189,14 @@
|
|||||||
used by Apache Ratis on datanodes.(128 MB by default)
|
used by Apache Ratis on datanodes.(128 MB by default)
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>dfs.container.ratis.statemachine.cache.expiry.interval</name>
|
||||||
|
<value>10s</value>
|
||||||
|
<tag>OZONE, RATIS, PERFORMANCE</tag>
|
||||||
|
<description>The interval till which the stateMachine data in ratis
|
||||||
|
will be cached inside the ContainerStateMachine.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>dfs.ratis.client.request.timeout.duration</name>
|
<name>dfs.ratis.client.request.timeout.duration</name>
|
||||||
<value>3s</value>
|
<value>3s</value>
|
||||||
|
@ -19,6 +19,8 @@
|
|||||||
package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.cache.Cache;
|
||||||
|
import com.google.common.cache.CacheBuilder;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
||||||
@ -27,6 +29,7 @@
|
|||||||
import org.apache.ratis.protocol.RaftGroupId;
|
import org.apache.ratis.protocol.RaftGroupId;
|
||||||
import org.apache.ratis.server.RaftServer;
|
import org.apache.ratis.server.RaftServer;
|
||||||
import org.apache.ratis.server.impl.RaftServerConstants;
|
import org.apache.ratis.server.impl.RaftServerConstants;
|
||||||
|
import org.apache.ratis.server.impl.RaftServerProxy;
|
||||||
import org.apache.ratis.server.protocol.TermIndex;
|
import org.apache.ratis.server.protocol.TermIndex;
|
||||||
import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
||||||
import org.apache.ratis.thirdparty.com.google.protobuf
|
import org.apache.ratis.thirdparty.com.google.protobuf
|
||||||
@ -60,12 +63,16 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
|
/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
|
||||||
@ -116,12 +123,11 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||||||
private final XceiverServerRatis ratisServer;
|
private final XceiverServerRatis ratisServer;
|
||||||
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
||||||
writeChunkFutureMap;
|
writeChunkFutureMap;
|
||||||
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
|
||||||
createContainerFutureMap;
|
|
||||||
private ExecutorService[] executors;
|
private ExecutorService[] executors;
|
||||||
private final int numExecutors;
|
private final int numExecutors;
|
||||||
private final Map<Long, Long> applyTransactionCompletionMap;
|
private final Map<Long, Long> applyTransactionCompletionMap;
|
||||||
private long lastIndex;
|
private long lastIndex;
|
||||||
|
private final Cache<Long, ByteString> stateMachineDataCache;
|
||||||
/**
|
/**
|
||||||
* CSM metrics.
|
* CSM metrics.
|
||||||
*/
|
*/
|
||||||
@ -129,7 +135,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||||||
|
|
||||||
public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
|
public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
|
||||||
ThreadPoolExecutor chunkExecutor, XceiverServerRatis ratisServer,
|
ThreadPoolExecutor chunkExecutor, XceiverServerRatis ratisServer,
|
||||||
List<ExecutorService> executors) {
|
List<ExecutorService> executors, long expiryInterval) {
|
||||||
this.gid = gid;
|
this.gid = gid;
|
||||||
this.dispatcher = dispatcher;
|
this.dispatcher = dispatcher;
|
||||||
this.chunkExecutor = chunkExecutor;
|
this.chunkExecutor = chunkExecutor;
|
||||||
@ -138,9 +144,13 @@ public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
|
|||||||
this.numExecutors = executors.size();
|
this.numExecutors = executors.size();
|
||||||
this.executors = executors.toArray(new ExecutorService[numExecutors]);
|
this.executors = executors.toArray(new ExecutorService[numExecutors]);
|
||||||
this.writeChunkFutureMap = new ConcurrentHashMap<>();
|
this.writeChunkFutureMap = new ConcurrentHashMap<>();
|
||||||
this.createContainerFutureMap = new ConcurrentHashMap<>();
|
|
||||||
applyTransactionCompletionMap = new ConcurrentHashMap<>();
|
applyTransactionCompletionMap = new ConcurrentHashMap<>();
|
||||||
this.lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
this.lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
||||||
|
stateMachineDataCache = CacheBuilder.newBuilder()
|
||||||
|
.expireAfterAccess(expiryInterval, TimeUnit.MILLISECONDS)
|
||||||
|
// set the limit on no of cached entries equal to no of max threads
|
||||||
|
// executing writeStateMachineData
|
||||||
|
.maximumSize(chunkExecutor.getCorePoolSize()).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -257,14 +267,6 @@ public TransactionContext startTransaction(RaftClientRequest request)
|
|||||||
.setStateMachineData(dataContainerCommandProto.toByteString())
|
.setStateMachineData(dataContainerCommandProto.toByteString())
|
||||||
.setLogData(commitContainerCommandProto.toByteString())
|
.setLogData(commitContainerCommandProto.toByteString())
|
||||||
.build();
|
.build();
|
||||||
} else if (proto.getCmdType() == Type.CreateContainer) {
|
|
||||||
return TransactionContext.newBuilder()
|
|
||||||
.setClientRequest(request)
|
|
||||||
.setStateMachine(this)
|
|
||||||
.setServerRole(RaftPeerRole.LEADER)
|
|
||||||
.setStateMachineData(request.getMessage().getContent())
|
|
||||||
.setLogData(request.getMessage().getContent())
|
|
||||||
.build();
|
|
||||||
} else {
|
} else {
|
||||||
return TransactionContext.newBuilder()
|
return TransactionContext.newBuilder()
|
||||||
.setClientRequest(request)
|
.setClientRequest(request)
|
||||||
@ -310,17 +312,17 @@ private ExecutorService getCommandExecutor(
|
|||||||
private CompletableFuture<Message> handleWriteChunk(
|
private CompletableFuture<Message> handleWriteChunk(
|
||||||
ContainerCommandRequestProto requestProto, long entryIndex) {
|
ContainerCommandRequestProto requestProto, long entryIndex) {
|
||||||
final WriteChunkRequestProto write = requestProto.getWriteChunk();
|
final WriteChunkRequestProto write = requestProto.getWriteChunk();
|
||||||
long containerID = write.getBlockID().getContainerID();
|
RaftServer server = ratisServer.getServer();
|
||||||
CompletableFuture<Message> future =
|
Preconditions.checkState(server instanceof RaftServerProxy);
|
||||||
createContainerFutureMap.get(containerID);
|
try {
|
||||||
CompletableFuture<Message> writeChunkFuture;
|
if (((RaftServerProxy) server).getImpl(gid).isLeader()) {
|
||||||
if (future != null) {
|
stateMachineDataCache.put(entryIndex, write.getData());
|
||||||
writeChunkFuture = future.thenApplyAsync(
|
|
||||||
v -> runCommand(requestProto), chunkExecutor);
|
|
||||||
} else {
|
|
||||||
writeChunkFuture = CompletableFuture.supplyAsync(
|
|
||||||
() -> runCommand(requestProto), chunkExecutor);
|
|
||||||
}
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
return completeExceptionally(ioe);
|
||||||
|
}
|
||||||
|
CompletableFuture<Message> writeChunkFuture = CompletableFuture
|
||||||
|
.supplyAsync(() -> runCommand(requestProto), chunkExecutor);
|
||||||
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
|
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
|
||||||
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
|
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
|
||||||
+ " logIndex " + entryIndex + " chunkName " + write.getChunkData()
|
+ " logIndex " + entryIndex + " chunkName " + write.getChunkData()
|
||||||
@ -337,14 +339,6 @@ private CompletableFuture<Message> handleWriteChunk(
|
|||||||
return writeChunkFuture;
|
return writeChunkFuture;
|
||||||
}
|
}
|
||||||
|
|
||||||
private CompletableFuture<Message> handleCreateContainer(
|
|
||||||
ContainerCommandRequestProto requestProto) {
|
|
||||||
long containerID = requestProto.getContainerID();
|
|
||||||
createContainerFutureMap.
|
|
||||||
computeIfAbsent(containerID, k -> new CompletableFuture<>());
|
|
||||||
return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* writeStateMachineData calls are not synchronized with each other
|
* writeStateMachineData calls are not synchronized with each other
|
||||||
* and also with applyTransaction.
|
* and also with applyTransaction.
|
||||||
@ -356,9 +350,10 @@ public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
|
|||||||
final ContainerCommandRequestProto requestProto =
|
final ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(getStateMachineData(entry.getStateMachineLogEntry()));
|
getRequestProto(getStateMachineData(entry.getStateMachineLogEntry()));
|
||||||
Type cmdType = requestProto.getCmdType();
|
Type cmdType = requestProto.getCmdType();
|
||||||
|
|
||||||
|
// For only writeChunk, there will be writeStateMachineData call.
|
||||||
|
// CreateContainer will happen as a part of writeChunk only.
|
||||||
switch (cmdType) {
|
switch (cmdType) {
|
||||||
case CreateContainer:
|
|
||||||
return handleCreateContainer(requestProto);
|
|
||||||
case WriteChunk:
|
case WriteChunk:
|
||||||
return handleWriteChunk(requestProto, entry.getIndex());
|
return handleWriteChunk(requestProto, entry.getIndex());
|
||||||
default:
|
default:
|
||||||
@ -397,7 +392,10 @@ private ByteString readStateMachineData(ContainerCommandRequestProto
|
|||||||
ReadChunkRequestProto.Builder readChunkRequestProto =
|
ReadChunkRequestProto.Builder readChunkRequestProto =
|
||||||
ReadChunkRequestProto.newBuilder()
|
ReadChunkRequestProto.newBuilder()
|
||||||
.setBlockID(writeChunkRequestProto.getBlockID())
|
.setBlockID(writeChunkRequestProto.getBlockID())
|
||||||
.setChunkData(writeChunkRequestProto.getChunkData());
|
.setChunkData(writeChunkRequestProto.getChunkData())
|
||||||
|
// set readFromTempFile to true in case, the chunkFile does
|
||||||
|
// not exist as applyTransaction is not executed for this entry yet.
|
||||||
|
.setReadFromTmpFile(true);
|
||||||
ContainerCommandRequestProto dataContainerCommandProto =
|
ContainerCommandRequestProto dataContainerCommandProto =
|
||||||
ContainerCommandRequestProto.newBuilder(requestProto)
|
ContainerCommandRequestProto.newBuilder(requestProto)
|
||||||
.setCmdType(Type.ReadChunk)
|
.setCmdType(Type.ReadChunk)
|
||||||
@ -409,15 +407,39 @@ private ByteString readStateMachineData(ContainerCommandRequestProto
|
|||||||
dispatchCommand(dataContainerCommandProto);
|
dispatchCommand(dataContainerCommandProto);
|
||||||
ReadChunkResponseProto responseProto = response.getReadChunk();
|
ReadChunkResponseProto responseProto = response.getReadChunk();
|
||||||
|
|
||||||
|
ByteString data = responseProto.getData();
|
||||||
// assert that the response has data in it.
|
// assert that the response has data in it.
|
||||||
Preconditions.checkNotNull(responseProto.getData());
|
Preconditions.checkNotNull(data);
|
||||||
|
return data;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reads the Entry from the Cache or loads it back by reading from disk.
|
||||||
|
*/
|
||||||
|
private ByteString getCachedStateMachineData(Long logIndex,
|
||||||
|
ContainerCommandRequestProto requestProto) throws ExecutionException {
|
||||||
|
try {
|
||||||
|
return reconstructWriteChunkRequest(
|
||||||
|
stateMachineDataCache.get(logIndex, new Callable<ByteString>() {
|
||||||
|
@Override
|
||||||
|
public ByteString call() throws Exception {
|
||||||
|
return readStateMachineData(requestProto);
|
||||||
|
}
|
||||||
|
}), requestProto);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private ByteString reconstructWriteChunkRequest(ByteString data,
|
||||||
|
ContainerCommandRequestProto requestProto) {
|
||||||
|
WriteChunkRequestProto writeChunkRequestProto =
|
||||||
|
requestProto.getWriteChunk();
|
||||||
// reconstruct the write chunk request
|
// reconstruct the write chunk request
|
||||||
final WriteChunkRequestProto.Builder dataWriteChunkProto =
|
final WriteChunkRequestProto.Builder dataWriteChunkProto =
|
||||||
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
|
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
|
||||||
// adding the state machine data
|
// adding the state machine data
|
||||||
.setData(responseProto.getData())
|
.setData(data).setStage(Stage.WRITE_DATA);
|
||||||
.setStage(Stage.WRITE_DATA);
|
|
||||||
|
|
||||||
ContainerCommandRequestProto.Builder newStateMachineProto =
|
ContainerCommandRequestProto.Builder newStateMachineProto =
|
||||||
ContainerCommandRequestProto.newBuilder(requestProto)
|
ContainerCommandRequestProto.newBuilder(requestProto)
|
||||||
@ -455,18 +477,21 @@ public CompletableFuture<ByteString> readStateMachineData(
|
|||||||
if (!getStateMachineData(smLogEntryProto).isEmpty()) {
|
if (!getStateMachineData(smLogEntryProto).isEmpty()) {
|
||||||
return CompletableFuture.completedFuture(ByteString.EMPTY);
|
return CompletableFuture.completedFuture(ByteString.EMPTY);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final ContainerCommandRequestProto requestProto =
|
final ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(entry.getStateMachineLogEntry().getLogData());
|
getRequestProto(entry.getStateMachineLogEntry().getLogData());
|
||||||
// readStateMachineData should only be called for "write" to Ratis.
|
// readStateMachineData should only be called for "write" to Ratis.
|
||||||
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
|
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
|
||||||
|
|
||||||
if (requestProto.getCmdType() == Type.WriteChunk) {
|
if (requestProto.getCmdType() == Type.WriteChunk) {
|
||||||
return CompletableFuture.supplyAsync(() ->
|
CompletableFuture<ByteString> future = new CompletableFuture<>();
|
||||||
readStateMachineData(requestProto), chunkExecutor);
|
return future.supplyAsync(() -> {
|
||||||
} else if (requestProto.getCmdType() == Type.CreateContainer) {
|
try {
|
||||||
return CompletableFuture.completedFuture(requestProto.toByteString());
|
return getCachedStateMachineData(entry.getIndex(), requestProto);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
future.completeExceptionally(e);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}, chunkExecutor);
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
|
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
|
||||||
+ " cannot have state machine data");
|
+ " cannot have state machine data");
|
||||||
@ -559,19 +584,6 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
|
|||||||
future = CompletableFuture.supplyAsync(() -> runCommand(requestProto),
|
future = CompletableFuture.supplyAsync(() -> runCommand(requestProto),
|
||||||
getCommandExecutor(requestProto));
|
getCommandExecutor(requestProto));
|
||||||
}
|
}
|
||||||
// Mark the createContainerFuture complete so that writeStateMachineData
|
|
||||||
// for WriteChunk gets unblocked
|
|
||||||
if (cmdType == Type.CreateContainer) {
|
|
||||||
long containerID = requestProto.getContainerID();
|
|
||||||
future.thenApply(
|
|
||||||
r -> {
|
|
||||||
createContainerFutureMap.remove(containerID).complete(null);
|
|
||||||
LOG.info("create Container Transaction completed for container " +
|
|
||||||
containerID + " log index " + index);
|
|
||||||
return r;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
lastIndex = index;
|
lastIndex = index;
|
||||||
future.thenAccept(m -> {
|
future.thenAccept(m -> {
|
||||||
final Long previous =
|
final Long previous =
|
||||||
@ -593,6 +605,11 @@ private static <T> CompletableFuture<T> completeExceptionally(Exception e) {
|
|||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void evictStateMachineCache() {
|
||||||
|
stateMachineDataCache.invalidateAll();
|
||||||
|
stateMachineDataCache.cleanUp();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void notifySlowness(RaftGroup group, RoleInfoProto roleInfoProto) {
|
public void notifySlowness(RaftGroup group, RoleInfoProto roleInfoProto) {
|
||||||
ratisServer.handleNodeSlowness(group, roleInfoProto);
|
ratisServer.handleNodeSlowness(group, roleInfoProto);
|
||||||
@ -604,7 +621,14 @@ public void notifyExtendedNoLeader(RaftGroup group,
|
|||||||
ratisServer.handleNoLeader(group, roleInfoProto);
|
ratisServer.handleNoLeader(group, roleInfoProto);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void notifyNotLeader(Collection<TransactionContext> pendingEntries)
|
||||||
|
throws IOException {
|
||||||
|
evictStateMachineCache();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
|
evictStateMachineCache();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -105,6 +105,8 @@ private static long nextCallId() {
|
|||||||
private final StateContext context;
|
private final StateContext context;
|
||||||
private final ReplicationLevel replicationLevel;
|
private final ReplicationLevel replicationLevel;
|
||||||
private long nodeFailureTimeoutMs;
|
private long nodeFailureTimeoutMs;
|
||||||
|
private final long cacheEntryExpiryInteval;
|
||||||
|
|
||||||
|
|
||||||
private XceiverServerRatis(DatanodeDetails dd, int port,
|
private XceiverServerRatis(DatanodeDetails dd, int port,
|
||||||
ContainerDispatcher dispatcher, Configuration conf, StateContext context)
|
ContainerDispatcher dispatcher, Configuration conf, StateContext context)
|
||||||
@ -128,6 +130,11 @@ private XceiverServerRatis(DatanodeDetails dd, int port,
|
|||||||
conf.getEnum(OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY,
|
conf.getEnum(OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY,
|
||||||
OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_DEFAULT);
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_DEFAULT);
|
||||||
this.executors = new ArrayList<>();
|
this.executors = new ArrayList<>();
|
||||||
|
cacheEntryExpiryInteval = conf.getTimeDuration(OzoneConfigKeys.
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL,
|
||||||
|
OzoneConfigKeys.
|
||||||
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
this.dispatcher = dispatcher;
|
this.dispatcher = dispatcher;
|
||||||
for (int i = 0; i < numContainerOpExecutors; i++) {
|
for (int i = 0; i < numContainerOpExecutors; i++) {
|
||||||
executors.add(Executors.newSingleThreadExecutor());
|
executors.add(Executors.newSingleThreadExecutor());
|
||||||
@ -141,8 +148,8 @@ private XceiverServerRatis(DatanodeDetails dd, int port,
|
|||||||
}
|
}
|
||||||
|
|
||||||
private ContainerStateMachine getStateMachine(RaftGroupId gid) {
|
private ContainerStateMachine getStateMachine(RaftGroupId gid) {
|
||||||
return new ContainerStateMachine(gid, dispatcher, chunkExecutor,
|
return new ContainerStateMachine(gid, dispatcher, chunkExecutor, this,
|
||||||
this, Collections.unmodifiableList(executors));
|
Collections.unmodifiableList(executors), cacheEntryExpiryInteval);
|
||||||
}
|
}
|
||||||
|
|
||||||
private RaftProperties newRaftProperties(Configuration conf) {
|
private RaftProperties newRaftProperties(Configuration conf) {
|
||||||
@ -304,6 +311,9 @@ private RaftProperties newRaftProperties(Configuration conf) {
|
|||||||
RaftServerConfigKeys.Log.StateMachineData.setSyncTimeoutRetry(properties,
|
RaftServerConfigKeys.Log.StateMachineData.setSyncTimeoutRetry(properties,
|
||||||
numSyncRetries);
|
numSyncRetries);
|
||||||
|
|
||||||
|
// Enable the StateMachineCaching
|
||||||
|
RaftServerConfigKeys.Log.StateMachineData
|
||||||
|
.setCachingEnabled(properties, true);
|
||||||
return properties;
|
return properties;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -531,7 +531,8 @@ ContainerCommandResponseProto handleReadChunk(
|
|||||||
.getChunkData());
|
.getChunkData());
|
||||||
Preconditions.checkNotNull(chunkInfo);
|
Preconditions.checkNotNull(chunkInfo);
|
||||||
|
|
||||||
data = chunkManager.readChunk(kvContainer, blockID, chunkInfo);
|
data = chunkManager.readChunk(kvContainer, blockID, chunkInfo,
|
||||||
|
request.getReadChunk().getReadFromTmpFile());
|
||||||
metrics.incContainerBytesStats(Type.ReadChunk, data.length);
|
metrics.incContainerBytesStats(Type.ReadChunk, data.length);
|
||||||
} catch (StorageContainerException ex) {
|
} catch (StorageContainerException ex) {
|
||||||
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
||||||
@ -702,8 +703,10 @@ ContainerCommandResponseProto handleGetSmallFile(
|
|||||||
ContainerProtos.ChunkInfo chunkInfo = null;
|
ContainerProtos.ChunkInfo chunkInfo = null;
|
||||||
ByteString dataBuf = ByteString.EMPTY;
|
ByteString dataBuf = ByteString.EMPTY;
|
||||||
for (ContainerProtos.ChunkInfo chunk : responseData.getChunks()) {
|
for (ContainerProtos.ChunkInfo chunk : responseData.getChunks()) {
|
||||||
|
// if the block is committed, all chunks must have been committed.
|
||||||
|
// Tmp chunk files won't exist here.
|
||||||
byte[] data = chunkManager.readChunk(kvContainer, blockID,
|
byte[] data = chunkManager.readChunk(kvContainer, blockID,
|
||||||
ChunkInfo.getFromProtoBuf(chunk));
|
ChunkInfo.getFromProtoBuf(chunk), false);
|
||||||
ByteString current = ByteString.copyFrom(data);
|
ByteString current = ByteString.copyFrom(data);
|
||||||
dataBuf = dataBuf.concat(current);
|
dataBuf = dataBuf.concat(current);
|
||||||
chunkInfo = chunk;
|
chunkInfo = chunk;
|
||||||
|
@ -173,13 +173,14 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
|
|||||||
* @param container - Container for the chunk
|
* @param container - Container for the chunk
|
||||||
* @param blockID - ID of the block.
|
* @param blockID - ID of the block.
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
|
* @param readFromTmpFile whether to read from tmp chunk file or not.
|
||||||
* @return byte array
|
* @return byte array
|
||||||
* @throws StorageContainerException
|
* @throws StorageContainerException
|
||||||
* TODO: Right now we do not support partial reads and writes of chunks.
|
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||||
* TODO: Explore if we need to do that for ozone.
|
* TODO: Explore if we need to do that for ozone.
|
||||||
*/
|
*/
|
||||||
public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info)
|
public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info,
|
||||||
throws StorageContainerException {
|
boolean readFromTmpFile) throws StorageContainerException {
|
||||||
try {
|
try {
|
||||||
KeyValueContainerData containerData = (KeyValueContainerData) container
|
KeyValueContainerData containerData = (KeyValueContainerData) container
|
||||||
.getContainerData();
|
.getContainerData();
|
||||||
@ -194,6 +195,12 @@ public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info)
|
|||||||
if (containerData.getLayOutVersion() == ChunkLayOutVersion
|
if (containerData.getLayOutVersion() == ChunkLayOutVersion
|
||||||
.getLatestVersion().getVersion()) {
|
.getLatestVersion().getVersion()) {
|
||||||
File chunkFile = ChunkUtils.getChunkFile(containerData, info);
|
File chunkFile = ChunkUtils.getChunkFile(containerData, info);
|
||||||
|
|
||||||
|
// In case the chunk file does not exist but tmp chunk file exist,
|
||||||
|
// read from tmp chunk file if readFromTmpFile is set to true
|
||||||
|
if (!chunkFile.exists() && readFromTmpFile) {
|
||||||
|
chunkFile = getTmpChunkFile(chunkFile, info);
|
||||||
|
}
|
||||||
data = ChunkUtils.readData(chunkFile, info, volumeIOStats);
|
data = ChunkUtils.readData(chunkFile, info, volumeIOStats);
|
||||||
containerData.incrReadCount();
|
containerData.incrReadCount();
|
||||||
long length = chunkFile.length();
|
long length = chunkFile.length();
|
||||||
|
@ -51,14 +51,15 @@ void writeChunk(Container container, BlockID blockID, ChunkInfo info,
|
|||||||
* @param container - Container for the chunk
|
* @param container - Container for the chunk
|
||||||
* @param blockID - ID of the block.
|
* @param blockID - ID of the block.
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
|
* @param readFromTmpFile whether to read from tmp chunk file or not
|
||||||
* @return byte array
|
* @return byte array
|
||||||
* @throws StorageContainerException
|
* @throws StorageContainerException
|
||||||
*
|
*
|
||||||
* TODO: Right now we do not support partial reads and writes of chunks.
|
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||||
* TODO: Explore if we need to do that for ozone.
|
* TODO: Explore if we need to do that for ozone.
|
||||||
*/
|
*/
|
||||||
byte[] readChunk(Container container, BlockID blockID, ChunkInfo info) throws
|
byte[] readChunk(Container container, BlockID blockID, ChunkInfo info,
|
||||||
StorageContainerException;
|
boolean readFromTmpFile) throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes a given chunk.
|
* Deletes a given chunk.
|
||||||
|
@ -189,7 +189,7 @@ public void testReadChunk() throws Exception {
|
|||||||
checkWriteIOStats(data.length, 1);
|
checkWriteIOStats(data.length, 1);
|
||||||
checkReadIOStats(0, 0);
|
checkReadIOStats(0, 0);
|
||||||
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
|
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
|
||||||
chunkInfo);
|
chunkInfo, false);
|
||||||
assertEquals(expectedData.length, data.length);
|
assertEquals(expectedData.length, data.length);
|
||||||
assertTrue(Arrays.equals(expectedData, data));
|
assertTrue(Arrays.equals(expectedData, data));
|
||||||
checkReadIOStats(data.length, 1);
|
checkReadIOStats(data.length, 1);
|
||||||
@ -226,7 +226,7 @@ public void testReadChunkFileNotExists() throws Exception {
|
|||||||
try {
|
try {
|
||||||
// trying to read a chunk, where chunk file does not exist
|
// trying to read a chunk, where chunk file does not exist
|
||||||
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
|
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
|
||||||
chunkInfo);
|
chunkInfo, false);
|
||||||
fail("testReadChunkFileNotExists failed");
|
fail("testReadChunkFileNotExists failed");
|
||||||
} catch (StorageContainerException ex) {
|
} catch (StorageContainerException ex) {
|
||||||
GenericTestUtils.assertExceptionContains("Unable to find the chunk " +
|
GenericTestUtils.assertExceptionContains("Unable to find the chunk " +
|
||||||
@ -249,7 +249,7 @@ public void testWriteAndReadChunkMultipleTimes() throws Exception {
|
|||||||
for (int i=0; i<100; i++) {
|
for (int i=0; i<100; i++) {
|
||||||
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
||||||
.getLocalID(), i), 0, data.length);
|
.getLocalID(), i), 0, data.length);
|
||||||
chunkManager.readChunk(keyValueContainer, blockID, chunkInfo);
|
chunkManager.readChunk(keyValueContainer, blockID, chunkInfo, false);
|
||||||
}
|
}
|
||||||
checkReadIOStats(data.length*100, 100);
|
checkReadIOStats(data.length*100, 100);
|
||||||
assertTrue(hddsVolume.getVolumeIOStats().getReadTime() > 0);
|
assertTrue(hddsVolume.getVolumeIOStats().getReadTime() > 0);
|
||||||
|
@ -406,7 +406,7 @@ public void testWritReadManyChunks() throws IOException {
|
|||||||
for (int x = 0; x < chunkCount; x++) {
|
for (int x = 0; x < chunkCount; x++) {
|
||||||
String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
|
String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
|
||||||
ChunkInfo info = fileHashMap.get(fileName);
|
ChunkInfo info = fileHashMap.get(fileName);
|
||||||
byte[] data = chunkManager.readChunk(container, blockID, info);
|
byte[] data = chunkManager.readChunk(container, blockID, info, false);
|
||||||
ChecksumData checksumData = checksum.computeChecksum(data);
|
ChecksumData checksumData = checksum.computeChecksum(data);
|
||||||
Assert.assertEquals(info.getChecksumData(), checksumData);
|
Assert.assertEquals(info.getChecksumData(), checksumData);
|
||||||
}
|
}
|
||||||
@ -435,11 +435,11 @@ public void testPartialRead() throws Exception {
|
|||||||
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
|
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
|
||||||
COMBINED);
|
COMBINED);
|
||||||
|
|
||||||
byte[] readData = chunkManager.readChunk(container, blockID, info);
|
byte[] readData = chunkManager.readChunk(container, blockID, info, false);
|
||||||
assertTrue(Arrays.equals(data, readData));
|
assertTrue(Arrays.equals(data, readData));
|
||||||
|
|
||||||
ChunkInfo info2 = getChunk(blockID.getLocalID(), 0, start, length);
|
ChunkInfo info2 = getChunk(blockID.getLocalID(), 0, start, length);
|
||||||
byte[] readData2 = chunkManager.readChunk(container, blockID, info2);
|
byte[] readData2 = chunkManager.readChunk(container, blockID, info2, false);
|
||||||
assertEquals(length, readData2.length);
|
assertEquals(length, readData2.length);
|
||||||
assertTrue(Arrays.equals(
|
assertTrue(Arrays.equals(
|
||||||
Arrays.copyOfRange(data, start, start + length), readData2));
|
Arrays.copyOfRange(data, start, start + length), readData2));
|
||||||
@ -513,7 +513,8 @@ public void testMultipleWriteSingleRead() throws IOException,
|
|||||||
// Request to read the whole data in a single go.
|
// Request to read the whole data in a single go.
|
||||||
ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0,
|
ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0,
|
||||||
datalen * chunkCount);
|
datalen * chunkCount);
|
||||||
byte[] newdata = chunkManager.readChunk(container, blockID, largeChunk);
|
byte[] newdata =
|
||||||
|
chunkManager.readChunk(container, blockID, largeChunk, false);
|
||||||
MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
||||||
newSha.update(newdata);
|
newSha.update(newdata);
|
||||||
Assert.assertEquals(Hex.encodeHexString(oldSha.digest()),
|
Assert.assertEquals(Hex.encodeHexString(oldSha.digest()),
|
||||||
@ -543,7 +544,7 @@ public void testDeleteChunk() throws IOException,
|
|||||||
chunkManager.deleteChunk(container, blockID, info);
|
chunkManager.deleteChunk(container, blockID, info);
|
||||||
exception.expect(StorageContainerException.class);
|
exception.expect(StorageContainerException.class);
|
||||||
exception.expectMessage("Unable to find the chunk file.");
|
exception.expectMessage("Unable to find the chunk file.");
|
||||||
chunkManager.readChunk(container, blockID, info);
|
chunkManager.readChunk(container, blockID, info, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
Loading…
Reference in New Issue
Block a user