GOSSIP-44 Remove 4 byte header

This commit is contained in:
Edward Capriolo
2017-01-20 11:40:52 -05:00
parent 428c0573fb
commit b2af449074
7 changed files with 118 additions and 156 deletions

View File

@ -31,7 +31,7 @@ public class StandAloneNode {
GossipSettings s = new GossipSettings(); GossipSettings s = new GossipSettings();
s.setWindowSize(10); s.setWindowSize(10);
s.setConvictThreshold(1.0); s.setConvictThreshold(1.0);
s.setGossipInterval(10); s.setGossipInterval(1000);
GossipService gossipService = new GossipService("mycluster", URI.create(args[0]), args[1], GossipService gossipService = new GossipService("mycluster", URI.create(args[0]), args[1],
Arrays.asList( new RemoteGossipMember("mycluster", URI.create(args[2]), args[3])), s, (a,b) -> {}, new MetricRegistry()); Arrays.asList( new RemoteGossipMember("mycluster", URI.create(args[2]), args[3])), s, (a,b) -> {}, new MetricRegistry());
gossipService.start(); gossipService.start();

View File

@ -17,7 +17,6 @@
*/ */
package org.apache.gossip.manager; package org.apache.gossip.manager;
import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -44,8 +43,6 @@ import org.apache.gossip.udp.UdpGossipDataMessage;
import org.apache.gossip.udp.UdpSharedGossipDataMessage; import org.apache.gossip.udp.UdpSharedGossipDataMessage;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import com.fasterxml.jackson.databind.ObjectMapper;
import static com.codahale.metrics.MetricRegistry.name; import static com.codahale.metrics.MetricRegistry.name;
/** /**
@ -61,7 +58,6 @@ public class ActiveGossipThread {
private ScheduledExecutorService scheduledExecutorService; private ScheduledExecutorService scheduledExecutorService;
private final BlockingQueue<Runnable> workQueue; private final BlockingQueue<Runnable> workQueue;
private ThreadPoolExecutor threadService; private ThreadPoolExecutor threadService;
private ObjectMapper MAPPER = new ObjectMapper();
private final Histogram sharedDataHistogram; private final Histogram sharedDataHistogram;
private final Histogram sendPerNodeDataHistogram; private final Histogram sendPerNodeDataHistogram;
@ -114,28 +110,17 @@ public class ActiveGossipThread {
LOGGER.debug("Send sendMembershipList() is called without action"); LOGGER.debug("Send sendMembershipList() is called without action");
sharedDataHistogram.update(System.currentTimeMillis() - startTime); sharedDataHistogram.update(System.currentTimeMillis() - startTime);
return; return;
} }
try { for (Entry<String, SharedGossipDataMessage> innerEntry : gossipCore.getSharedData().entrySet()){
for (Entry<String, SharedGossipDataMessage> innerEntry : this.gossipCore.getSharedData().entrySet()){ UdpSharedGossipDataMessage message = new UdpSharedGossipDataMessage();
UdpSharedGossipDataMessage message = new UdpSharedGossipDataMessage(); message.setUuid(UUID.randomUUID().toString());
message.setUuid(UUID.randomUUID().toString()); message.setUriFrom(me.getId());
message.setUriFrom(me.getId()); message.setExpireAt(innerEntry.getValue().getExpireAt());
message.setExpireAt(innerEntry.getValue().getExpireAt()); message.setKey(innerEntry.getValue().getKey());
message.setKey(innerEntry.getValue().getKey()); message.setNodeId(innerEntry.getValue().getNodeId());
message.setNodeId(innerEntry.getValue().getNodeId()); message.setTimestamp(innerEntry.getValue().getTimestamp());
message.setTimestamp(innerEntry.getValue().getTimestamp()); message.setPayload(innerEntry.getValue().getPayload());
message.setPayload(innerEntry.getValue().getPayload()); gossipCore.sendOneWay(message, member.getUri());
byte[] json_bytes = MAPPER.writeValueAsString(message).getBytes();
int packet_length = json_bytes.length;
if (packet_length < GossipManager.MAX_PACKET_SIZE) {
gossipCore.sendOneWay(message, member.getUri());
} else {
LOGGER.error("The length of the to be send message is too large ("
+ packet_length + " > " + GossipManager.MAX_PACKET_SIZE + ").");
}
}
} catch (IOException e1) {
LOGGER.warn(e1);
} }
sharedDataHistogram.update(System.currentTimeMillis() - startTime); sharedDataHistogram.update(System.currentTimeMillis() - startTime);
} }
@ -148,36 +133,26 @@ public class ActiveGossipThread {
LOGGER.debug("Send sendMembershipList() is called without action"); LOGGER.debug("Send sendMembershipList() is called without action");
sendPerNodeDataHistogram.update(System.currentTimeMillis() - startTime); sendPerNodeDataHistogram.update(System.currentTimeMillis() - startTime);
return; return;
} }
try { for (Entry<String, ConcurrentHashMap<String, GossipDataMessage>> entry : gossipCore.getPerNodeData().entrySet()){
for (Entry<String, ConcurrentHashMap<String, GossipDataMessage>> entry : gossipCore.getPerNodeData().entrySet()){ for (Entry<String, GossipDataMessage> innerEntry : entry.getValue().entrySet()){
for (Entry<String, GossipDataMessage> innerEntry : entry.getValue().entrySet()){ UdpGossipDataMessage message = new UdpGossipDataMessage();
UdpGossipDataMessage message = new UdpGossipDataMessage(); message.setUuid(UUID.randomUUID().toString());
message.setUuid(UUID.randomUUID().toString()); message.setUriFrom(me.getId());
message.setUriFrom(me.getId()); message.setExpireAt(innerEntry.getValue().getExpireAt());
message.setExpireAt(innerEntry.getValue().getExpireAt()); message.setKey(innerEntry.getValue().getKey());
message.setKey(innerEntry.getValue().getKey()); message.setNodeId(innerEntry.getValue().getNodeId());
message.setNodeId(innerEntry.getValue().getNodeId()); message.setTimestamp(innerEntry.getValue().getTimestamp());
message.setTimestamp(innerEntry.getValue().getTimestamp()); message.setPayload(innerEntry.getValue().getPayload());
message.setPayload(innerEntry.getValue().getPayload()); gossipCore.sendOneWay(message, member.getUri());
byte[] json_bytes = MAPPER.writeValueAsString(message).getBytes();
int packet_length = json_bytes.length;
if (packet_length < GossipManager.MAX_PACKET_SIZE) {
gossipCore.sendOneWay(message, member.getUri());
} else {
LOGGER.error("The length of the to be send message is too large ("
+ packet_length + " > " + GossipManager.MAX_PACKET_SIZE + ").");
}
}
} }
} catch (IOException e1) {
LOGGER.warn(e1);
} }
sendPerNodeDataHistogram.update(System.currentTimeMillis() - startTime); sendPerNodeDataHistogram.update(System.currentTimeMillis() - startTime);
} }
protected void sendToALiveMember(){ protected void sendToALiveMember(){
LocalGossipMember member = selectPartner(gossipManager.getLiveMembers()); LocalGossipMember member = selectPartner(gossipManager.getLiveMembers());
System.out.println("send" );
sendMembershipList(gossipManager.getMyself(), member); sendMembershipList(gossipManager.getMyself(), member);
} }
@ -199,29 +174,18 @@ public class ActiveGossipThread {
} else { } else {
LOGGER.debug("Send sendMembershipList() is called to " + member.toString()); LOGGER.debug("Send sendMembershipList() is called to " + member.toString());
} }
try { UdpActiveGossipMessage message = new UdpActiveGossipMessage();
UdpActiveGossipMessage message = new UdpActiveGossipMessage(); message.setUriFrom(gossipManager.getMyself().getUri().toASCIIString());
message.setUriFrom(gossipManager.getMyself().getUri().toASCIIString()); message.setUuid(UUID.randomUUID().toString());
message.setUuid(UUID.randomUUID().toString()); message.getMembers().add(convert(me));
message.getMembers().add(convert(me)); for (LocalGossipMember other : gossipManager.getMembers().keySet()) {
for (LocalGossipMember other : gossipManager.getMembers().keySet()) { message.getMembers().add(convert(other));
message.getMembers().add(convert(other)); }
} Response r = gossipCore.send(message, member.getUri());
byte[] json_bytes = MAPPER.writeValueAsString(message).getBytes(); if (r instanceof ActiveGossipOk){
int packet_length = json_bytes.length; //maybe count metrics here
if (packet_length < GossipManager.MAX_PACKET_SIZE) { } else {
Response r = gossipCore.send(message, member.getUri()); LOGGER.debug("Message " + message + " generated response " + r);
if (r instanceof ActiveGossipOk){
//maybe count metrics here
} else {
LOGGER.debug("Message " + message + " generated response " + r);
}
} else {
LOGGER.error("The length of the to be send message is too large ("
+ packet_length + " > " + GossipManager.MAX_PACKET_SIZE + ").");
}
} catch (IOException e1) {
LOGGER.warn(e1);
} }
sendMembershipHistorgram.update(System.currentTimeMillis() - startTime); sendMembershipHistorgram.update(System.currentTimeMillis() - startTime);
} }

View File

@ -31,7 +31,6 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -53,27 +52,42 @@ import org.apache.gossip.udp.UdpNotAMemberFault;
import org.apache.gossip.udp.UdpSharedGossipDataMessage; import org.apache.gossip.udp.UdpSharedGossipDataMessage;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricRegistry;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
public class GossipCore { public class GossipCore implements GossipCoreConstants {
public static final Logger LOGGER = Logger.getLogger(GossipCore.class); public static final Logger LOGGER = Logger.getLogger(GossipCore.class);
private static final ObjectMapper MAPPER = new ObjectMapper(); private static final ObjectMapper MAPPER = new ObjectMapper();
private final GossipManager gossipManager; private final GossipManager gossipManager;
private ConcurrentHashMap<String, Base> requests; private ConcurrentHashMap<String, Base> requests;
private ExecutorService service; private ThreadPoolExecutor service;
private final ConcurrentHashMap<String, ConcurrentHashMap<String, GossipDataMessage>> perNodeData; private final ConcurrentHashMap<String, ConcurrentHashMap<String, GossipDataMessage>> perNodeData;
private final ConcurrentHashMap<String, SharedGossipDataMessage> sharedData; private final ConcurrentHashMap<String, SharedGossipDataMessage> sharedData;
private final BlockingQueue<Runnable> workQueue; private final BlockingQueue<Runnable> workQueue;
private final Meter messageSerdeException;
private final Meter tranmissionException;
private final Meter tranmissionSuccess;
public GossipCore(GossipManager manager){ public GossipCore(GossipManager manager, MetricRegistry metrics){
this.gossipManager = manager; this.gossipManager = manager;
requests = new ConcurrentHashMap<>(); requests = new ConcurrentHashMap<>();
workQueue = new ArrayBlockingQueue<>(1024); workQueue = new ArrayBlockingQueue<>(1024);
service = new ThreadPoolExecutor(1, 5, 1, TimeUnit.SECONDS, workQueue, new ThreadPoolExecutor.DiscardOldestPolicy()); service = new ThreadPoolExecutor(1, 5, 1, TimeUnit.SECONDS, workQueue, new ThreadPoolExecutor.DiscardOldestPolicy());
perNodeData = new ConcurrentHashMap<>(); perNodeData = new ConcurrentHashMap<>();
sharedData = new ConcurrentHashMap<>(); sharedData = new ConcurrentHashMap<>();
metrics.register(WORKQUEUE_SIZE, (Gauge<Integer>)() -> workQueue.size());
metrics.register(PER_NODE_DATA_SIZE, (Gauge<Integer>)() -> perNodeData.size());
metrics.register(SHARED_DATA_SIZE, (Gauge<Integer>)() -> sharedData.size());
metrics.register(REQUEST_SIZE, (Gauge<Integer>)() -> requests.size());
metrics.register(THREADPOOL_ACTIVE, (Gauge<Integer>)() -> service.getActiveCount());
metrics.register(THREADPOOL_SIZE, (Gauge<Integer>)() -> service.getPoolSize());
messageSerdeException = metrics.meter(MESSAGE_SERDE_EXCEPTION);
tranmissionException = metrics.meter(MESSAGE_TRANSMISSION_EXCEPTION);
tranmissionSuccess = metrics.meter(MESSAGE_TRANSMISSION_SUCCESS);
} }
public void addSharedData(SharedGossipDataMessage message){ public void addSharedData(SharedGossipDataMessage message){
@ -175,29 +189,29 @@ public class GossipCore {
} }
/** /**
* Sends a blocking message. Throws exception when tranmission fails * Sends a blocking message.
* @param message * @param message
* @param uri * @param uri
* @throws RuntimeException if data can not be serialized or in transmission error
*/ */
private void sendInternal(Base message, URI uri){ private void sendInternal(Base message, URI uri){
byte[] json_bytes; byte[] json_bytes;
try { try {
json_bytes = MAPPER.writeValueAsString(message).getBytes(); json_bytes = MAPPER.writeValueAsString(message).getBytes();
} catch (IOException e) { } catch (IOException e) {
messageSerdeException.mark();
throw new RuntimeException(e); throw new RuntimeException(e);
} }
int packet_length = json_bytes.length; try (DatagramSocket socket = new DatagramSocket()) {
if (packet_length < GossipManager.MAX_PACKET_SIZE) { socket.setSoTimeout(gossipManager.getSettings().getGossipInterval() * 2);
byte[] buf = UdpUtil.createBuffer(packet_length, json_bytes); InetAddress dest = InetAddress.getByName(uri.getHost());
try (DatagramSocket socket = new DatagramSocket()) { DatagramPacket datagramPacket = new DatagramPacket(json_bytes, json_bytes.length, dest, uri.getPort());
socket.setSoTimeout(gossipManager.getSettings().getGossipInterval() * 2); socket.send(datagramPacket);
InetAddress dest = InetAddress.getByName(uri.getHost()); tranmissionSuccess.mark();
DatagramPacket datagramPacket = new DatagramPacket(buf, buf.length, dest, uri.getPort()); } catch (IOException e) {
socket.send(datagramPacket); tranmissionException.mark();
} catch (IOException e) { throw new RuntimeException(e);
throw new RuntimeException(e); }
}
}
} }
public Response send(Base message, URI uri){ public Response send(Base message, URI uri){
@ -225,7 +239,7 @@ public class GossipCore {
return (Response) b; return (Response) b;
} }
try { try {
Thread.sleep(0, 1000); Thread.sleep(0, 555555);
} catch (InterruptedException e) { } catch (InterruptedException e) {
} }
@ -261,19 +275,20 @@ public class GossipCore {
public void sendOneWay(Base message, URI u){ public void sendOneWay(Base message, URI u){
byte[] json_bytes; byte[] json_bytes;
try { try {
json_bytes = MAPPER.writeValueAsString(message).getBytes(); json_bytes = MAPPER.writeValueAsBytes(message);
} catch (IOException e) { } catch (IOException e) {
messageSerdeException.mark();
throw new RuntimeException(e); throw new RuntimeException(e);
} }
int packet_length = json_bytes.length; try (DatagramSocket socket = new DatagramSocket()) {
if (packet_length < GossipManager.MAX_PACKET_SIZE) { socket.setSoTimeout(gossipManager.getSettings().getGossipInterval() * 2);
byte[] buf = UdpUtil.createBuffer(packet_length, json_bytes); InetAddress dest = InetAddress.getByName(u.getHost());
try (DatagramSocket socket = new DatagramSocket()) { DatagramPacket datagramPacket = new DatagramPacket(json_bytes, json_bytes.length, dest, u.getPort());
socket.setSoTimeout(gossipManager.getSettings().getGossipInterval() * 2); socket.send(datagramPacket);
InetAddress dest = InetAddress.getByName(u.getHost()); tranmissionSuccess.mark();
DatagramPacket datagramPacket = new DatagramPacket(buf, buf.length, dest, u.getPort()); } catch (IOException ex) {
socket.send(datagramPacket); tranmissionException.mark();
} catch (IOException ex) { } LOGGER.debug("Send one way failed", ex);
} }
} }

View File

@ -17,29 +17,14 @@
*/ */
package org.apache.gossip.manager; package org.apache.gossip.manager;
import java.nio.ByteBuffer; public interface GossipCoreConstants {
String WORKQUEUE_SIZE = "gossip.core.workqueue.size";
public class UdpUtil { String PER_NODE_DATA_SIZE = "gossip.core.pernodedata.size";
String SHARED_DATA_SIZE = "gossip.core.shareddata.size";
public static int readPacketLengthFromBuffer(byte [] buffer){ String REQUEST_SIZE = "gossip.core.requests.size";
int packetLength = 0; String THREADPOOL_ACTIVE = "gossip.core.threadpool.active";
for (int i = 0; i < 4; i++) { String THREADPOOL_SIZE = "gossip.core.threadpool.size";
int shift = (4 - 1 - i) * 8; String MESSAGE_SERDE_EXCEPTION = "gossip.core.message_serde_exception";
packetLength += (buffer[i] & 0x000000FF) << shift; String MESSAGE_TRANSMISSION_EXCEPTION = "gossip.core.message_transmission_exception";
} String MESSAGE_TRANSMISSION_SUCCESS = "gossip.core.message_transmission_success";
return packetLength;
}
public static byte[] createBuffer(int packetLength, byte[] jsonBytes) {
byte[] lengthBytes = new byte[4];
lengthBytes[0] = (byte) (packetLength >> 24);
lengthBytes[1] = (byte) ((packetLength << 8) >> 24);
lengthBytes[2] = (byte) ((packetLength << 16) >> 24);
lengthBytes[3] = (byte) ((packetLength << 24) >> 24);
ByteBuffer byteBuffer = ByteBuffer.allocate(4 + jsonBytes.length);
byteBuffer.put(lengthBytes);
byteBuffer.put(jsonBytes);
byte[] buf = byteBuffer.array();
return buf;
}
} }

View File

@ -49,8 +49,6 @@ public abstract class GossipManager {
public static final Logger LOGGER = Logger.getLogger(GossipManager.class); public static final Logger LOGGER = Logger.getLogger(GossipManager.class);
public static final int MAX_PACKET_SIZE = 102400;
private final ConcurrentSkipListMap<LocalGossipMember, GossipState> members; private final ConcurrentSkipListMap<LocalGossipMember, GossipState> members;
private final LocalGossipMember me; private final LocalGossipMember me;
@ -82,7 +80,7 @@ public abstract class GossipManager {
List<GossipMember> gossipMembers, GossipListener listener, MetricRegistry registry) { List<GossipMember> gossipMembers, GossipListener listener, MetricRegistry registry) {
this.settings = settings; this.settings = settings;
gossipCore = new GossipCore(this); gossipCore = new GossipCore(this, registry);
clock = new SystemClock(); clock = new SystemClock();
dataReaper = new DataReaper(gossipCore, clock); dataReaper = new DataReaper(gossipCore, clock);
me = new LocalGossipMember(cluster, uri, id, clock.nanoTime(), me = new LocalGossipMember(cluster, uri, id, clock.nanoTime(),
@ -256,4 +254,5 @@ public abstract class GossipManager {
return dataReaper; return dataReaper;
} }
} }

View File

@ -78,23 +78,13 @@ abstract public class PassiveGossipThread implements Runnable {
byte[] buf = new byte[server.getReceiveBufferSize()]; byte[] buf = new byte[server.getReceiveBufferSize()];
DatagramPacket p = new DatagramPacket(buf, buf.length); DatagramPacket p = new DatagramPacket(buf, buf.length);
server.receive(p); server.receive(p);
int packet_length = UdpUtil.readPacketLengthFromBuffer(buf); debug(p.getData());
if (packet_length <= GossipManager.MAX_PACKET_SIZE) { try {
byte[] json_bytes = new byte[packet_length]; Base activeGossipMessage = MAPPER.readValue(p.getData(), Base.class);
for (int i = 0; i < packet_length; i++) { gossipCore.receive(activeGossipMessage);
json_bytes[i] = buf[i + 4]; } catch (RuntimeException ex) {//TODO trap json exception
} LOGGER.error("Unable to process message", ex);
debug(packet_length, json_bytes);
try {
Base activeGossipMessage = MAPPER.readValue(json_bytes, Base.class);
gossipCore.receive(activeGossipMessage);
} catch (RuntimeException ex) {//TODO trap json exception
LOGGER.error("Unable to process message", ex);
}
} else {
LOGGER.error("The received message is not of the expected size, it has been dropped.");
} }
} catch (IOException e) { } catch (IOException e) {
LOGGER.error(e); LOGGER.error(e);
keepRunning.set(false); keepRunning.set(false);
@ -103,11 +93,10 @@ abstract public class PassiveGossipThread implements Runnable {
shutdown(); shutdown();
} }
private void debug(int packetLength, byte[] jsonBytes) { private void debug(byte[] jsonBytes) {
if (LOGGER.isDebugEnabled()){ if (LOGGER.isDebugEnabled()){
String receivedMessage = new String(jsonBytes); String receivedMessage = new String(jsonBytes);
LOGGER.debug("Received message (" + packetLength + " bytes): " LOGGER.debug("Received message ( bytes): " + receivedMessage);
+ receivedMessage);
} }
} }

View File

@ -32,27 +32,37 @@ import io.teknek.tunit.TUnit;
public class DataReaperTest { public class DataReaperTest {
private final MetricRegistry registry = new MetricRegistry(); private final MetricRegistry registry = new MetricRegistry();
String myId = "4";
String key = "key";
String value = "a";
@Test @Test
public void testReaperOneShot() { public void testReaperOneShot() {
String myId = "4";
String key = "key";
String value = "a";
GossipSettings settings = new GossipSettings(); GossipSettings settings = new GossipSettings();
GossipManager gm = RandomGossipManager.newBuilder().cluster("abc").settings(settings) GossipManager gm = RandomGossipManager.newBuilder().cluster("abc").settings(settings)
.withId(myId).uri(URI.create("udp://localhost:6000")).registry(registry).build(); .withId(myId).uri(URI.create("udp://localhost:6000")).registry(registry).build();
gm.init(); gm.init();
gm.gossipPerNodeData(perNodeDatum(key, value)); gm.gossipPerNodeData(perNodeDatum(key, value));
gm.gossipSharedData(sharedDatum(key, value)); gm.gossipSharedData(sharedDatum(key, value));
Assert.assertEquals(value, gm.findPerNodeGossipData(myId, key).getPayload()); assertDataIsAtCorrectValue(gm);
Assert.assertEquals(value, gm.findSharedGossipData(key).getPayload());
gm.getDataReaper().runPerNodeOnce(); gm.getDataReaper().runPerNodeOnce();
gm.getDataReaper().runSharedOnce(); gm.getDataReaper().runSharedOnce();
TUnit.assertThat(() -> gm.findPerNodeGossipData(myId, key)).equals(null); assertDataIsRemoved(gm);
TUnit.assertThat(() -> gm.findSharedGossipData(key)).equals(null);
gm.shutdown(); gm.shutdown();
} }
private void assertDataIsAtCorrectValue(GossipManager gm){
Assert.assertEquals(value, gm.findPerNodeGossipData(myId, key).getPayload());
Assert.assertEquals(1, registry.getGauges().get(GossipCoreConstants.PER_NODE_DATA_SIZE).getValue());
Assert.assertEquals(value, gm.findSharedGossipData(key).getPayload());
Assert.assertEquals(1, registry.getGauges().get(GossipCoreConstants.SHARED_DATA_SIZE).getValue());
}
private void assertDataIsRemoved(GossipManager gm){
TUnit.assertThat(() -> gm.findPerNodeGossipData(myId, key)).equals(null);
TUnit.assertThat(() -> gm.findSharedGossipData(key)).equals(null);
}
private GossipDataMessage perNodeDatum(String key, String value) { private GossipDataMessage perNodeDatum(String key, String value) {
GossipDataMessage m = new GossipDataMessage(); GossipDataMessage m = new GossipDataMessage();
m.setExpireAt(System.currentTimeMillis() + 5L); m.setExpireAt(System.currentTimeMillis() + 5L);