Skip to content

Ratis 952. Avoid NPE #461

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 12 additions & 12 deletions ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
Expand All @@ -51,9 +52,8 @@ static InterruptedIOException toInterruptedIOException(
}

static IOException asIOException(Throwable t) {
return t == null? null
: t instanceof IOException? (IOException)t
: new IOException(t);
Objects.requireNonNull(t, "t == null");
return t instanceof IOException? (IOException)t : new IOException(t);
}

static IOException toIOException(ExecutionException e) {
Expand Down Expand Up @@ -115,7 +115,8 @@ static void readFully(InputStream in, int buffSize) throws IOException {
*/
static void readFully(InputStream in, byte[] buf, int off, int len)
throws IOException {
for(int toRead = len; toRead > 0; ) {
int toRead = len;
while (toRead > 0) {
final int ret = in.read(buf, off, toRead);
if (ret < 0) {
final int read = len - toRead;
Expand Down Expand Up @@ -148,7 +149,7 @@ static long preallocate(FileChannel fc, long size, ByteBuffer fill) throws IOExc
final int remaining = fill.remaining();

long allocated = 0;
for(; allocated < size; ) {
while (allocated < size) {
final long required = size - allocated;
final int n = remaining < required? remaining: Math.toIntExact(required);
final ByteBuffer buffer = fill.slice();
Expand Down Expand Up @@ -215,14 +216,13 @@ static <T> T bytes2Object(byte[] bytes, Class<T> clazz) {
}

static <T> T readObject(InputStream in, Class<T> clazz) {
Object obj = null;
try(ObjectInputStream oin = new ObjectInputStream(in)) {
final Object obj = oin.readObject();
try {
return clazz.cast(obj);
} catch (ClassCastException e) {
throw new IllegalStateException("Failed to cast to " + clazz + ", object="
+ (obj instanceof Throwable? StringUtils.stringifyException((Throwable) obj): obj), e);
}
obj = oin.readObject();
return clazz.cast(obj);
} catch (ClassCastException e) {
throw new IllegalStateException("Failed to cast to " + clazz + ", object="
+ (obj instanceof Throwable? StringUtils.stringifyException((Throwable) obj): obj), e);
} catch (IOException | ClassNotFoundException e) {
throw new IllegalStateException("Failed to read an object.", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@

import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.ratis.client.impl.ClientProtoUtils;
import org.apache.ratis.proto.hadoop.HadoopCompatibilityProtos.ClientReplyProto;
Expand Down Expand Up @@ -53,7 +52,6 @@ public CombinedClientProtocolServerSideTranslatorPB(RaftServer impl) {
}

@Override
@SuppressFBWarnings("NP_NULL_ON_SOME_PATH")
public ClientReplyProto sendClient(RpcController unused, ClientRequestProto req) throws ServiceException {
ByteBuffer buf = req.getRequest().asReadOnlyByteBuffer();
GeneratedMessageV3 response = null;
Expand All @@ -79,6 +77,7 @@ public ClientReplyProto sendClient(RpcController unused, ClientRequestProto req)
response = transferLeadership(TransferLeadershipRequestProto.parseFrom(buf));
break;
default:
throw new ServiceException("Internal error, all response types are not being handled as expected.");
}
} catch(IOException ioe) {
throw new ServiceException(ioe);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,14 +128,14 @@ public void setProperties(RaftProperties properties) {
}

@Override
@SuppressFBWarnings("NP_NULL_ON_SOME_PATH")
public TransactionContext applyTransactionSerial(TransactionContext trx) {
public TransactionContext applyTransactionSerial(TransactionContext trx) throws InvalidProtocolBufferException {
RaftProtos.LogEntryProto x = trx.getLogEntry();
MetaSMRequestProto req = null;
try {
req = MetaSMRequestProto.parseFrom(x.getStateMachineLogEntry().getLogData());
} catch (InvalidProtocolBufferException e) {
e.printStackTrace();
throw e;
}
switch (req.getTypeCase()) {
case REGISTERREQUEST:
Expand Down Expand Up @@ -203,47 +203,52 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
}

@Override
@SuppressFBWarnings("NP_NULL_ON_SOME_PATH")
public CompletableFuture<Message> query(Message request) {
Timer.Context timerContext = null;
MetaServiceProtos.MetaServiceRequestProto.TypeCase type = null;
try {
if (currentGroup == null) {
try {
List<RaftGroup> x =
StreamSupport.stream(raftServer.getGroups().spliterator(), false)
.filter(group -> group.getGroupId().equals(metadataGroupId)).collect(Collectors.toList());
if (x.size() == 1) {
currentGroup = x.get(0);
}
} catch (IOException e) {
e.printStackTrace();
}
}

MetaServiceProtos.MetaServiceRequestProto req = null;
if (currentGroup == null) {
try {
req = MetaServiceProtos.MetaServiceRequestProto.parseFrom(request.getContent());
} catch (InvalidProtocolBufferException e) {
List<RaftGroup> x =
StreamSupport.stream(raftServer.getGroups().spliterator(), false)
.filter(group -> group.getGroupId().equals(metadataGroupId)).collect(Collectors.toList());
if (x.size() == 1) {
currentGroup = x.get(0);
}
} catch (IOException e) {
e.printStackTrace();
}
type = req.getTypeCase();
}

MetaServiceProtos.MetaServiceRequestProto req = null;
try {
req = MetaServiceProtos.MetaServiceRequestProto.parseFrom(request.getContent());
} catch (InvalidProtocolBufferException e) {
e.printStackTrace();
return null;
}
type = req.getTypeCase();
// Main purpose of this try catch block is to make sure that
// timerContext.stop() is run after return.
try {
timerContext = logServiceMetaDataMetrics.getTimer(type.name()).time();
switch (type) {

case CREATELOG:
return processCreateLogRequest(req);
case LISTLOGS:
return processListLogsRequest();
case GETLOG:
return processGetLogRequest(req);
case DELETELOG:
return processDeleteLog(req);
default:
case CREATELOG:
return processCreateLogRequest(req);
case LISTLOGS:
return processListLogsRequest();
case GETLOG:
return processGetLogRequest(req);
case DELETELOG:
return processDeleteLog(req);
default:
CompletableFuture<Message> reply = super.query(request);
return reply;
}
CompletableFuture<Message> reply = super.query(request);
return reply;
}finally{
} catch (Exception e) {
LOG.error("Exception during Meta State Machine query");
throw e;
} finally {
if (timerContext != null) {
timerContext.stop();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.storage.RaftStorage;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.util.JavaUtils;
import org.apache.ratis.util.LifeCycle;
import org.slf4j.Logger;
Expand Down Expand Up @@ -444,7 +445,7 @@ default FollowerEventApi followerEvent() {
* of the raft peers
* @return The Transaction context.
*/
TransactionContext applyTransactionSerial(TransactionContext trx);
TransactionContext applyTransactionSerial(TransactionContext trx) throws InvalidProtocolBufferException;

/**
* Apply a committed log entry to the state machine. This method is called sequentially in
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,9 @@ boolean containsInConf(RaftPeerId peerId) {

boolean isHighestPriority(RaftPeerId peerId) {
RaftPeer target = getPeer(peerId);
if (target == null) {
return false;
}
Collection<RaftPeer> peers = getCurrentPeers();
for (RaftPeer peer : peers) {
if (peer.getPriority() >= target.getPriority() && !peer.equals(target)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.LogProtoUtils;
import org.apache.ratis.server.raftlog.RaftLog;
import org.apache.ratis.server.raftlog.RaftLogIOException;
import org.apache.ratis.server.storage.RaftStorage;
import org.apache.ratis.server.storage.RaftStorageDirectory;
import org.apache.ratis.server.util.ServerStringUtils;
Expand Down Expand Up @@ -827,10 +828,15 @@ private CompletableFuture<RaftClientReply> streamAsync(RaftClientRequest request
newExceptionReply(request, generateNotLeaderException())));
}

private CompletableFuture<RaftClientRequest> streamEndOfRequestAsync(RaftClientRequest request) {
CompletableFuture<RaftClientRequest> streamEndOfRequestAsync(RaftClientRequest request) {
return role.getLeaderState()
.map(ls -> ls.streamEndOfRequestAsync(request))
.orElse(null);
.orElseGet(() -> {
final CompletableFuture<RaftClientRequest> errorF = new CompletableFuture<>();
errorF.completeExceptionally(
new Exception("Unexpected null encountered, while receiving end of stream request."));
return errorF;
});
}

CompletableFuture<RaftClientReply> processQueryFuture(
Expand Down Expand Up @@ -1604,7 +1610,7 @@ private CompletableFuture<Message> replyPendingRequest(
// update the retry cache
final CacheEntry cacheEntry = retryCache.getOrCreateEntry(invocationId);
if (getInfo().isLeader()) {
Preconditions.assertTrue(cacheEntry != null && !cacheEntry.isCompletedNormally(),
Preconditions.assertTrue(!cacheEntry.isCompletedNormally(),
"retry cache entry should be pending: %s", cacheEntry);
}
if (cacheEntry.isFailed()) {
Expand Down Expand Up @@ -1634,7 +1640,7 @@ private CompletableFuture<Message> replyPendingRequest(
});
}

CompletableFuture<Message> applyLogToStateMachine(LogEntryProto next) {
CompletableFuture<Message> applyLogToStateMachine(LogEntryProto next) throws RaftLogIOException {
if (!next.hasStateMachineLogEntry()) {
stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex());
}
Expand All @@ -1654,16 +1660,14 @@ CompletableFuture<Message> applyLogToStateMachine(LogEntryProto next) {
.setLogEntry(next)
.build());

// Let the StateMachine inject logic for committed transactions in sequential order.
trx = stateMachine.applyTransactionSerial(trx);

try {
// Let the StateMachine inject logic for committed transactions in sequential order.
trx = stateMachine.applyTransactionSerial(trx);

final CompletableFuture<Message> stateMachineFuture = stateMachine.applyTransaction(trx);
return replyPendingRequest(next, stateMachineFuture);
} catch (Exception e) {
LOG.error("{}: applyTransaction failed for index:{} proto:{}",
getMemberId(), next.getIndex(), LogProtoUtils.toLogEntryString(next), e);
throw e;
throw new RaftLogIOException(e);
}
}
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,13 +181,11 @@ public String toString() {
}

CacheEntry getOrCreateEntry(ClientInvocationId key) {
final CacheEntry entry;
try {
entry = cache.get(key, () -> new CacheEntry(key));
return cache.get(key, () -> new CacheEntry(key));
} catch (ExecutionException e) {
throw new IllegalStateException(e);
}
return entry;
}

CacheEntry refreshEntry(CacheEntry newEntry) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.statemachine.StateMachineStorage;
import org.apache.ratis.statemachine.TransactionContext;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.util.JavaUtils;
import org.apache.ratis.util.LifeCycle;
import org.apache.ratis.util.Preconditions;
Expand Down Expand Up @@ -104,7 +105,7 @@ public void reinitialize() throws IOException {
}

@Override
public TransactionContext applyTransactionSerial(TransactionContext trx) {
public TransactionContext applyTransactionSerial(TransactionContext trx) throws InvalidProtocolBufferException {
return trx;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -274,6 +274,11 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except
}
}

// Wait until index has been updated
JavaUtils.attemptUntilTrue(
() -> cluster.getLeader().getStateMachine().getLastAppliedTermIndex().getIndex() == 20,
10, ONE_SECOND, "leaderIndexIsCorrect", LOG);

// Take snapshot and check result.
long snapshotIndex = cluster.getLeader().getStateMachine().takeSnapshot();
Assert.assertEquals(20, snapshotIndex);
Expand Down Expand Up @@ -330,6 +335,11 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except

// Make sure leader and followers are still up to date.
for (RaftServer.Division follower : cluster.getFollowers()) {
long leaderIndex = leader.getRaftLog().getNextIndex();
// Give follower slightly time to catch up
JavaUtils.attemptUntilTrue(
() -> leaderIndex == follower.getRaftLog().getNextIndex(),
10, ONE_SECOND, "leaderAndFollowerIndexMatch", LOG);
Assert.assertEquals(
leader.getRaftLog().getNextIndex(),
follower.getRaftLog().getNextIndex());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.ratis.server.impl;

import org.apache.ratis.protocol.*;
import org.apache.ratis.server.*;

public class RaftConfigurationTestUtil {
public static boolean isHighestPriority(RaftConfiguration config, RaftPeerId peerId) {
return ((RaftConfigurationImpl)config).isHighestPriority(peerId);
}
}
Loading