Skip to content

Commit

Permalink
Improve the performance of CAS operation
Browse files Browse the repository at this point in the history
Combine the paxos prepare and quorum read stage together
  • Loading branch information
DikangGu committed Jun 8, 2018
1 parent d3b6a67 commit da43bb7
Show file tree
Hide file tree
Showing 11 changed files with 500 additions and 42 deletions.
17 changes: 13 additions & 4 deletions src/java/org/apache/cassandra/net/MessagingService.java
Expand Up @@ -104,6 +104,8 @@
import org.apache.cassandra.service.AbstractWriteResponseHandler;
import org.apache.cassandra.service.StorageProxy;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.service.paxos.PrepareAndReadCommand;
import org.apache.cassandra.service.paxos.PrepareAndReadResponse;
import org.apache.cassandra.service.paxos.Commit;
import org.apache.cassandra.service.paxos.PrepareResponse;
import org.apache.cassandra.tracing.TraceState;
Expand Down Expand Up @@ -261,14 +263,20 @@ public long getTimeout()
return DatabaseDescriptor.getPingTimeout();
}
},
PAXOS_PREPARE_AND_READ
{
public long getTimeout()
{
return DatabaseDescriptor.getReadRpcTimeout();
}
},

// UNUSED verbs were used as padding for backward/forward compatability before 4.0,
// but it wasn't quite as bullet/future proof as needed. We still need to keep these entries
// around, at least for a major rev or two (post-4.0). see CASSANDRA-13993 for a discussion.
// For now, though, the UNUSED are legacy values (placeholders, basically) that should only be used
// for correctly adding VERBs that need to be emergency additions to 3.0/3.11.
// We can reclaim them (their id's, to be correct) in future versions, if desireed, though.
UNUSED_2,
UNUSED_3,
UNUSED_4,
UNUSED_5,
Expand Down Expand Up @@ -359,13 +367,12 @@ public static Verb fromId(int id)
put(Verb.REPLICATION_FINISHED, Stage.MISC);
put(Verb.SNAPSHOT, Stage.MISC);
put(Verb.ECHO, Stage.GOSSIP);
put(Verb.PING, Stage.READ);
put(Verb.PAXOS_PREPARE_AND_READ, Stage.MUTATION);

put(Verb.UNUSED_2, Stage.INTERNAL_RESPONSE);
put(Verb.UNUSED_3, Stage.INTERNAL_RESPONSE);
put(Verb.UNUSED_4, Stage.INTERNAL_RESPONSE);
put(Verb.UNUSED_5, Stage.INTERNAL_RESPONSE);

put(Verb.PING, Stage.READ);
}};

/**
Expand Down Expand Up @@ -401,6 +408,7 @@ public static Verb fromId(int id)
put(Verb.PAXOS_PREPARE, Commit.serializer);
put(Verb.PAXOS_PROPOSE, Commit.serializer);
put(Verb.PAXOS_COMMIT, Commit.serializer);
put(Verb.PAXOS_PREPARE_AND_READ, PrepareAndReadCommand.serializer);
put(Verb.HINT, HintMessage.serializer);
put(Verb.BATCH_STORE, Batch.serializer);
put(Verb.BATCH_REMOVE, UUIDSerializer.serializer);
Expand Down Expand Up @@ -429,6 +437,7 @@ public static Verb fromId(int id)

put(Verb.PAXOS_PREPARE, PrepareResponse.serializer);
put(Verb.PAXOS_PROPOSE, BooleanSerializer.serializer);
put(Verb.PAXOS_PREPARE_AND_READ, PrepareAndReadResponse.serializer);

put(Verb.BATCH_STORE, WriteResponse.serializer);
put(Verb.BATCH_REMOVE, WriteResponse.serializer);
Expand Down
170 changes: 137 additions & 33 deletions src/java/org/apache/cassandra/service/StorageProxy.java

Large diffs are not rendered by default.

2 changes: 2 additions & 0 deletions src/java/org/apache/cassandra/service/StorageService.java
Expand Up @@ -99,6 +99,7 @@
import org.apache.cassandra.schema.TableMetadataRef;
import org.apache.cassandra.schema.ViewMetadata;
import org.apache.cassandra.repair.RepairMessageVerbHandler;
import org.apache.cassandra.service.paxos.PrepareAndReadVerbHandler;
import org.apache.cassandra.service.paxos.CommitVerbHandler;
import org.apache.cassandra.service.paxos.PrepareVerbHandler;
import org.apache.cassandra.service.paxos.ProposeVerbHandler;
Expand Down Expand Up @@ -288,6 +289,7 @@ public StorageService()
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_PREPARE, new PrepareVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_PROPOSE, new ProposeVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_COMMIT, new CommitVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_PREPARE_AND_READ, new PrepareAndReadVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.HINT, new HintVerbHandler());

// see BootStrapper for a summary of how the bootstrap verbs interact
Expand Down
2 changes: 2 additions & 0 deletions src/java/org/apache/cassandra/service/paxos/PaxosState.java
Expand Up @@ -32,6 +32,8 @@

public class PaxosState
{
public static final boolean USE_FASTPAXOS = Boolean.getBoolean("cassandra.use_fastpaxos");

private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentWriters() * 1024);

private final Commit promised;
Expand Down
@@ -0,0 +1,58 @@
/*
* 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.cassandra.service.paxos;

import org.apache.cassandra.db.partitions.PartitionIterator;
import org.apache.cassandra.net.IAsyncCallback;
import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.service.reads.DataResolver;
import org.apache.cassandra.service.reads.ReadCallback;

public class PrepareAndReadCallback implements IAsyncCallback<PrepareAndReadResponse>
{
public final PrepareCallback prepareCallback;
public final ReadCallback readCallback;
public PartitionIterator iterator = null;

public PrepareAndReadCallback(PrepareCallback prepareCallback, ReadCallback readCallback)
{
this.prepareCallback = prepareCallback;
this.readCallback = readCallback;
}

@Override
public void response(MessageIn<PrepareAndReadResponse> msg)
{
PrepareAndReadResponse response = msg.payload;
prepareCallback.response(response.prepareResponse, msg.from);
readCallback.response(response.readResponse);
}

@Override
public boolean isLatencyForSnitch()
{
return false;
}

public void await()
{
readCallback.awaitResults();
iterator = ((DataResolver)readCallback.resolver).getData();
}
}
@@ -0,0 +1,65 @@
/*
* 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.cassandra.service.paxos;

import java.io.IOException;

import org.apache.cassandra.db.ReadCommand;
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;

public class PrepareAndReadCommand
{
public static final CASPrepareCommandSerializer serializer = new CASPrepareCommandSerializer();

public final Commit commit;
public final ReadCommand readCommand;

public PrepareAndReadCommand(Commit commit, ReadCommand readCommand)
{
this.commit = commit;
this.readCommand = readCommand;
}

public static class CASPrepareCommandSerializer implements IVersionedSerializer<PrepareAndReadCommand>
{
@Override
public void serialize(PrepareAndReadCommand prepareAndReadCommand, DataOutputPlus out, int version) throws IOException
{
Commit.serializer.serialize(prepareAndReadCommand.commit, out, version);
ReadCommand.serializer.serialize(prepareAndReadCommand.readCommand, out, version);
}

@Override
public PrepareAndReadCommand deserialize(DataInputPlus in, int version) throws IOException
{
Commit commit = Commit.serializer.deserialize(in, version);
ReadCommand readCommand = ReadCommand.serializer.deserialize(in, version);
return new PrepareAndReadCommand(commit, readCommand);
}

@Override
public long serializedSize(PrepareAndReadCommand prepareAndReadCommand, int version)
{
return Commit.serializer.serializedSize(prepareAndReadCommand.commit, version) +
ReadCommand.serializer.serializedSize(prepareAndReadCommand.readCommand, version);
}
}
}
@@ -0,0 +1,68 @@
/*
* 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.cassandra.service.paxos;

import java.io.IOException;

import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;

public class PrepareAndReadResponse
{
public static final PrepareResponseSerializer serializer = new PrepareResponseSerializer();

public final PrepareResponse prepareResponse;
public final ReadResponse readResponse;

public PrepareAndReadResponse(PrepareResponse prepareResponse, ReadResponse readResponse)
{
this.prepareResponse = prepareResponse;
this.readResponse = readResponse;
}

@Override
public String toString()
{
return String.format("PrepareAndReadResponse(%s, %s)", prepareResponse, readResponse);
}

public static class PrepareResponseSerializer implements IVersionedSerializer<PrepareAndReadResponse>
{
public void serialize(PrepareAndReadResponse response, DataOutputPlus out, int version) throws IOException
{
ReadResponse.serializer.serialize(response.readResponse, out, version);
PrepareResponse.serializer.serialize(response.prepareResponse, out, version);
}

public PrepareAndReadResponse deserialize(DataInputPlus in, int version) throws IOException
{
ReadResponse readResponse = ReadResponse.serializer.deserialize(in, version);
PrepareResponse prepareResponse = PrepareResponse.serializer.deserialize(in, version);
return new PrepareAndReadResponse(prepareResponse, readResponse);
}

public long serializedSize(PrepareAndReadResponse response, int version)
{
return PrepareResponse.serializer.serializedSize(response.prepareResponse, version) +
ReadResponse.serializer.serializedSize(response.readResponse, version);
}
}
}
@@ -0,0 +1,54 @@
/*
* 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.cassandra.service.paxos;

import java.io.IOException;

import org.apache.cassandra.db.ReadCommand;
import org.apache.cassandra.db.ReadExecutionController;
import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.net.IVerbHandler;
import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.net.MessageOut;
import org.apache.cassandra.net.MessagingService;

public class PrepareAndReadVerbHandler implements IVerbHandler<PrepareAndReadCommand>
{
@Override
public void doVerb(MessageIn<PrepareAndReadCommand> message, int id) throws IOException
{
PrepareAndReadCommand prepareCommand = message.payload;
PrepareResponse paxosResponse = PaxosState.prepare(prepareCommand.commit);
ReadCommand command = prepareCommand.readCommand;

ReadResponse readResponse;

try (ReadExecutionController executionController = command.executionController();
UnfilteredPartitionIterator iterator = command.executeLocally(executionController))
{
readResponse = command.createResponse(iterator);
}

PrepareAndReadResponse response = new PrepareAndReadResponse(paxosResponse, readResponse);

MessageOut<PrepareAndReadResponse> reply = new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, response, PrepareAndReadResponse.serializer);
MessagingService.instance().sendReply(reply, id, message.from);
}
}
Expand Up @@ -64,6 +64,11 @@ public synchronized void response(MessageIn<PrepareResponse> message)
PrepareResponse response = message.payload;
logger.trace("Prepare response {} from {}", response, message.from);

response(response, message.from);
}

public synchronized void response(PrepareResponse response, InetAddressAndPort from)
{
// In case of clock skew, another node could be proposing with ballot that are quite a bit
// older than our own. In that case, we record the more recent commit we've received to make
// sure we re-prepare on an older ballot.
Expand All @@ -78,7 +83,7 @@ public synchronized void response(MessageIn<PrepareResponse> message)
return;
}

commitsByReplica.put(message.from, response.mostRecentCommit);
commitsByReplica.put(from, response.mostRecentCommit);
if (response.mostRecentCommit.isAfter(mostRecentCommit))
mostRecentCommit = response.mostRecentCommit;

Expand Down

0 comments on commit da43bb7

Please sign in to comment.