package org.apache.cassandra.gms;
import java.io.IOException;
import java.net.InetAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.net.CompactEndpointSerializationHelper;
public class GossipDigestAck
{
public static final IVersionedSerializer<GossipDigestAck> serializer = new GossipDigestAckSerializer();
final List<GossipDigest> gDigestList;
final Map<InetAddress, EndpointState> epStateMap;
GossipDigestAck(List<GossipDigest> gDigestList, Map<InetAddress, EndpointState> epStateMap)
{
this.gDigestList = gDigestList;
this.epStateMap = epStateMap;
}
List<GossipDigest> getGossipDigestList()
{
return gDigestList;
}
Map<InetAddress, EndpointState> getEndpointStateMap()
{
return epStateMap;
}
}
class GossipDigestAckSerializer implements IVersionedSerializer<GossipDigestAck>
{
public void serialize(GossipDigestAck gDigestAckMessage, DataOutputPlus out, int version) throws IOException
{
GossipDigestSerializationHelper.serialize(gDigestAckMessage.gDigestList, out, version);
out.writeInt(gDigestAckMessage.epStateMap.size());
for (Map.Entry<InetAddress, EndpointState> entry : gDigestAckMessage.epStateMap.entrySet())
{
InetAddress ep = entry.getKey();
CompactEndpointSerializationHelper.serialize(ep, out);
EndpointState.serializer.serialize(entry.getValue(), out, version);
}
}
public GossipDigestAck deserialize(DataInputPlus in, int version) throws IOException
{
List<GossipDigest> gDigestList = GossipDigestSerializationHelper.deserialize(in, version);
int size = in.readInt();
Map<InetAddress, EndpointState> epStateMap = new HashMap<InetAddress, EndpointState>(size);
for (int i = 0; i < size; ++i)
{
InetAddress ep = CompactEndpointSerializationHelper.deserialize(in);
EndpointState epState = EndpointState.serializer.deserialize(in, version);
epStateMap.put(ep, epState);
}
return new GossipDigestAck(gDigestList, epStateMap);
}
public long serializedSize(GossipDigestAck ack, int version)
{
int size = GossipDigestSerializationHelper.serializedSize(ack.gDigestList, version);
size += TypeSizes.sizeof(ack.epStateMap.size());
for (Map.Entry<InetAddress, EndpointState> entry : ack.epStateMap.entrySet())
size += CompactEndpointSerializationHelper.serializedSize(entry.getKey())
+ EndpointState.serializer.serializedSize(entry.getValue(), version);
return size;
}
}