You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2013/06/21 18:56:21 UTC
[1/3] git commit: Gossiper.handleMajorStateChange can lose existing
node ApplicationState patch by jasobrown;
reviewe4d by jbellis for CASSANDRA-5665
Updated Branches:
refs/heads/trunk 36aae612a -> d99a6f2a2
Gossiper.handleMajorStateChange can lose existing node ApplicationState
patch by jasobrown; reviewe4d by jbellis for CASSANDRA-5665
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d99a6f2a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d99a6f2a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d99a6f2a
Branch: refs/heads/trunk
Commit: d99a6f2a2bda290404c6706e7b1db87c59dd04cb
Parents: b8b96bd
Author: Jason Brown <ja...@gmail.com>
Authored: Fri Jun 21 09:23:17 2013 -0700
Committer: Jason Brown <ja...@gmail.com>
Committed: Fri Jun 21 09:55:34 2013 -0700
----------------------------------------------------------------------
src/java/org/apache/cassandra/gms/Gossiper.java | 45 +++++++++++---------
1 file changed, 25 insertions(+), 20 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d99a6f2a/src/java/org/apache/cassandra/gms/Gossiper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index d8918f3..d1a2ab5 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -874,6 +874,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
if (logger.isTraceEnabled())
logger.trace("Updating heartbeat state generation to " + remoteGeneration + " from " + localGeneration + " for " + ep);
// major state change will handle the update by inserting the remote state directly
+ copyNewerApplicationStates(remoteState, localEpStatePtr);
handleMajorStateChange(ep, remoteState);
}
else if (remoteGeneration == localGeneration) // generation has not changed, apply new states
@@ -883,8 +884,15 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
int remoteMaxVersion = getMaxEndpointStateVersion(remoteState);
if (remoteMaxVersion > localMaxVersion)
{
- // apply states, but do not notify since there is no major change
- applyNewStates(ep, localEpStatePtr, remoteState);
+ if (logger.isTraceEnabled())
+ {
+ logger.trace("Updating heartbeat state version to " + remoteState.getHeartBeatState().getHeartBeatVersion() +
+ " from " + localEpStatePtr.getHeartBeatState().getHeartBeatVersion() + " for " + ep);
+ }
+ localEpStatePtr.setHeartBeatState(remoteState.getHeartBeatState());
+ Map<ApplicationState, VersionedValue> merged = copyNewerApplicationStates(localEpStatePtr, remoteState);
+ for (Entry<ApplicationState, VersionedValue> appState : merged.entrySet())
+ doNotifications(ep, appState.getKey(), appState.getValue());
}
else if (logger.isTraceEnabled())
logger.trace("Ignoring remote version " + remoteMaxVersion + " <= " + localMaxVersion + " for " + ep);
@@ -906,28 +914,25 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
}
}
- private void applyNewStates(InetAddress addr, EndpointState localState, EndpointState remoteState)
+ private Map<ApplicationState, VersionedValue> copyNewerApplicationStates(EndpointState toState, EndpointState fromState)
{
- // don't assert here, since if the node restarts the version will go back to zero
- int oldVersion = localState.getHeartBeatState().getHeartBeatVersion();
-
- localState.setHeartBeatState(remoteState.getHeartBeatState());
- if (logger.isTraceEnabled())
- logger.trace("Updating heartbeat state version to " + localState.getHeartBeatState().getHeartBeatVersion() + " from " + oldVersion + " for " + addr + " ...");
-
- // we need to make two loops here, one to apply, then another to notify, this way all states in an update are present and current when the notifications are received
- for (Entry<ApplicationState, VersionedValue> remoteEntry : remoteState.getApplicationStateMap().entrySet())
+ Map<ApplicationState, VersionedValue> merged = new HashMap<ApplicationState, VersionedValue>();
+ for (Entry<ApplicationState, VersionedValue> fromEntry : fromState.getApplicationStateMap().entrySet())
{
- ApplicationState remoteKey = remoteEntry.getKey();
- VersionedValue remoteValue = remoteEntry.getValue();
+ ApplicationState key = fromEntry.getKey();
+ VersionedValue value = fromEntry.getValue();
+ assert fromState.getHeartBeatState().getGeneration() == toState.getHeartBeatState().getGeneration();
- assert remoteState.getHeartBeatState().getGeneration() == localState.getHeartBeatState().getGeneration();
- localState.addApplicationState(remoteKey, remoteValue);
- }
- for (Entry<ApplicationState, VersionedValue> remoteEntry : remoteState.getApplicationStateMap().entrySet())
- {
- doNotifications(addr, remoteEntry.getKey(), remoteEntry.getValue());
+ if ( (toState.applicationState.containsKey(key) && toState.applicationState.get(key).compareTo(value) < 0)
+ || !toState.applicationState.containsKey(key) )
+ {
+ if (logger.isTraceEnabled())
+ logger.trace("merging {}:{} into ApplicationState", key, value);
+ toState.addApplicationState(key, value);
+ merged.put(key, value);
+ }
}
+ return merged;
}
// notify that an application state has changed
[3/3] git commit: refactor reconnecting snitches patch by jasobrown;
reviewed by jbellis for CASSANDRA-5681
Posted by ja...@apache.org.
refactor reconnecting snitches
patch by jasobrown; reviewed by jbellis for CASSANDRA-5681
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ded233ef
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ded233ef
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ded233ef
Branch: refs/heads/trunk
Commit: ded233ef0c93ecfe0c1bf582cfa004e5ca574934
Parents: 36aae61
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Jun 21 10:40:31 2013 -0500
Committer: Jason Brown <ja...@gmail.com>
Committed: Fri Jun 21 09:55:34 2013 -0700
----------------------------------------------------------------------
.../cassandra/locator/Ec2MultiRegionSnitch.java | 71 +---------------
.../locator/GossipingPropertyFileSnitch.java | 63 +-------------
.../locator/ReconnectableSnitchHelper.java | 88 ++++++++++++++++++++
3 files changed, 94 insertions(+), 128 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/ded233ef/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
index 9317941..bd5e091 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -19,16 +19,11 @@ package org.apache.cassandra.locator;
import java.io.IOException;
import java.net.InetAddress;
-import java.net.UnknownHostException;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.EndpointState;
import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.service.StorageService;
/**
@@ -36,16 +31,13 @@ import org.apache.cassandra.service.StorageService;
*
* 2) Snitch will set the private IP as a Gossip application state.
*
- * 3) Snitch implements IESCS and will reset the connection if it is within the
+ * 3) Uses a helper class that implements IESCS and will reset the public IP connection if it is within the
* same region to communicate via private IP.
*
- * Implements Ec2Snitch to inherit its functionality and extend it for
- * Multi-Region.
- *
* Operational: All the nodes in this cluster needs to be able to (modify the
* Security group settings in AWS) communicate via Public IP's.
*/
-public class Ec2MultiRegionSnitch extends Ec2Snitch implements IEndpointStateChangeSubscriber
+public class Ec2MultiRegionSnitch extends Ec2Snitch
{
private static final String PUBLIC_IP_QUERY_URL = "http://169.254.169.254/latest/meta-data/public-ipv4";
private static final String PRIVATE_IP_QUERY_URL = "http://169.254.169.254/latest/meta-data/local-ipv4";
@@ -62,67 +54,10 @@ public class Ec2MultiRegionSnitch extends Ec2Snitch implements IEndpointStateCha
DatabaseDescriptor.setBroadcastAddress(localPublicAddress);
}
- public void onJoin(InetAddress endpoint, EndpointState epState)
- {
- if (epState.getApplicationState(ApplicationState.INTERNAL_IP) != null)
- reconnect(endpoint, epState.getApplicationState(ApplicationState.INTERNAL_IP));
- }
-
- public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
- {
- if (state == ApplicationState.INTERNAL_IP)
- reconnect(endpoint, value);
- }
-
- public void onAlive(InetAddress endpoint, EndpointState state)
- {
- if (state.getApplicationState(ApplicationState.INTERNAL_IP) != null)
- reconnect(endpoint, state.getApplicationState(ApplicationState.INTERNAL_IP));
- }
-
- public void onDead(InetAddress endpoint, EndpointState state)
- {
- // do nothing
- }
-
- public void onRestart(InetAddress endpoint, EndpointState state)
- {
- // do nothing
- }
-
- public void onRemove(InetAddress endpoint)
- {
- // do nothing.
- }
-
- private void reconnect(InetAddress publicAddress, VersionedValue localAddressValue)
- {
- try
- {
- reconnect(publicAddress, InetAddress.getByName(localAddressValue.value));
- }
- catch (UnknownHostException e)
- {
- logger.error("Error in getting the IP address resolved: ", e);
- }
- }
-
- private void reconnect(InetAddress publicAddress, InetAddress localAddress)
- {
- if (getDatacenter(publicAddress).equals(getDatacenter(localPublicAddress))
- && MessagingService.instance().getVersion(publicAddress) == MessagingService.current_version
- && !MessagingService.instance().getConnectionPool(publicAddress).endPoint().equals(localAddress))
- {
- MessagingService.instance().getConnectionPool(publicAddress).reset(localAddress);
- logger.debug(String.format("Intiated reconnect to an Internal IP %s for the %s", localAddress, publicAddress));
- }
- }
-
- @Override
public void gossiperStarting()
{
super.gossiperStarting();
Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, StorageService.instance.valueFactory.internalIP(localPrivateAddress));
- Gossiper.instance.register(this);
+ Gossiper.instance.register(new ReconnectableSnitchHelper(this, ec2region, true));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/ded233ef/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
index 071cd09..e00239e 100644
--- a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.locator;
import java.net.InetAddress;
-import java.net.UnknownHostException;
import java.util.Map;
import org.apache.cassandra.db.SystemTable;
@@ -30,14 +29,11 @@ import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.gms.ApplicationState;
import org.apache.cassandra.gms.EndpointState;
import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.service.StorageService;
-public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch implements IEndpointStateChangeSubscriber
+public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch// implements IEndpointStateChangeSubscriber
{
private static final Logger logger = LoggerFactory.getLogger(GossipingPropertyFileSnitch.class);
@@ -47,7 +43,7 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch i
private Map<InetAddress, Map<String, String>> savedEndpoints;
private String DEFAULT_DC = "UNKNOWN_DC";
private String DEFAULT_RACK = "UNKNOWN_RACK";
- private boolean preferLocal;
+ private final boolean preferLocal;
public GossipingPropertyFileSnitch() throws ConfigurationException
{
@@ -126,64 +122,11 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch i
return epState.getApplicationState(ApplicationState.RACK).value;
}
- // IEndpointStateChangeSubscriber methods
-
- public void onJoin(InetAddress endpoint, EndpointState epState)
- {
- if (preferLocal && epState.getApplicationState(ApplicationState.INTERNAL_IP) != null)
- reConnect(endpoint, epState.getApplicationState(ApplicationState.INTERNAL_IP));
- }
-
- public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
- {
- if (preferLocal && state == ApplicationState.INTERNAL_IP)
- reConnect(endpoint, value);
- }
-
- public void onAlive(InetAddress endpoint, EndpointState state)
- {
- if (preferLocal && state.getApplicationState(ApplicationState.INTERNAL_IP) != null)
- reConnect(endpoint, state.getApplicationState(ApplicationState.INTERNAL_IP));
- }
-
- public void onDead(InetAddress endpoint, EndpointState state)
- {
- // do nothing
- }
-
- public void onRestart(InetAddress endpoint, EndpointState state)
- {
- // do nothing
- }
-
- public void onRemove(InetAddress endpoint)
- {
- // do nothing.
- }
-
- private void reConnect(InetAddress endpoint, VersionedValue versionedValue)
- {
- if (!getDatacenter(endpoint).equals(myDC))
- return; // do nothing return back...
-
- try
- {
- InetAddress remoteIP = InetAddress.getByName(versionedValue.value);
- MessagingService.instance().getConnectionPool(endpoint).reset(remoteIP);
- logger.debug(String.format("Intiated reconnect to an Internal IP %s for the endpoint %s", remoteIP, endpoint));
- }
- catch (UnknownHostException e)
- {
- logger.error("Error in getting the IP address resolved", e);
- }
- }
-
- @Override
public void gossiperStarting()
{
super.gossiperStarting();
Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP,
StorageService.instance.valueFactory.internalIP(FBUtilities.getLocalAddress().getHostAddress()));
- Gossiper.instance.register(this);
+ Gossiper.instance.register(new ReconnectableSnitchHelper(this, myDC, preferLocal));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/ded233ef/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
new file mode 100644
index 0000000..adec953
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java
@@ -0,0 +1,88 @@
+package org.apache.cassandra.locator;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.gms.EndpointState;
+import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
+import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.net.MessagingService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Sidekick helper for snitches that want to reconnect from one IP addr for a node to another.
+ * Typically, this is for situations like EC2 where a node will have a public address and a private address,
+ * where we connect on the public, discover the private, and reconnect on the private.
+ */
+public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber
+{
+ private static final Logger logger = LoggerFactory.getLogger(ReconnectableSnitchHelper.class);
+ private final IEndpointSnitch snitch;
+ private final String localDc;
+ private final boolean preferLocal;
+
+ public ReconnectableSnitchHelper(IEndpointSnitch snitch, String localDc, boolean preferLocal)
+ {
+ this.snitch = snitch;
+ this.localDc = localDc;
+ this.preferLocal = preferLocal;
+ }
+
+ private void reconnect(InetAddress publicAddress, VersionedValue localAddressValue)
+ {
+ try
+ {
+ reconnect(publicAddress, InetAddress.getByName(localAddressValue.value));
+ }
+ catch (UnknownHostException e)
+ {
+ logger.error("Error in getting the IP address resolved: ", e);
+ }
+ }
+
+ private void reconnect(InetAddress publicAddress, InetAddress localAddress)
+ {
+ if (snitch.getDatacenter(publicAddress).equals(localDc)
+ && MessagingService.instance().getVersion(publicAddress) == MessagingService.current_version
+ && !MessagingService.instance().getConnectionPool(publicAddress).endPoint().equals(localAddress))
+ {
+ MessagingService.instance().getConnectionPool(publicAddress).reset(localAddress);
+ logger.debug(String.format("Intiated reconnect to an Internal IP %s for the %s", localAddress, publicAddress));
+ }
+ }
+
+ public void onJoin(InetAddress endpoint, EndpointState epState)
+ {
+ if (preferLocal && epState.getApplicationState(ApplicationState.INTERNAL_IP) != null)
+ reconnect(endpoint, epState.getApplicationState(ApplicationState.INTERNAL_IP));
+ }
+
+ public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
+ {
+ if (preferLocal && state == ApplicationState.INTERNAL_IP)
+ reconnect(endpoint, value);
+ }
+
+ public void onAlive(InetAddress endpoint, EndpointState state)
+ {
+ if (preferLocal && state.getApplicationState(ApplicationState.INTERNAL_IP) != null)
+ reconnect(endpoint, state.getApplicationState(ApplicationState.INTERNAL_IP));
+ }
+
+ public void onDead(InetAddress endpoint, EndpointState state)
+ {
+ // do nothing.
+ }
+
+ public void onRemove(InetAddress endpoint)
+ {
+ // do nothing.
+ }
+
+ public void onRestart(InetAddress endpoint, EndpointState state)
+ {
+ // do nothing.
+ }
+}
[2/3] git commit: restore fetching global trace state in default
.execute method
Posted by ja...@apache.org.
restore fetching global trace state in default .execute method
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b8b96bdb
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b8b96bdb
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b8b96bdb
Branch: refs/heads/trunk
Commit: b8b96bdb733aed87197df3f01834fe47cc756ae8
Parents: ded233e
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Jun 21 11:13:21 2013 -0500
Committer: Jason Brown <ja...@gmail.com>
Committed: Fri Jun 21 09:55:34 2013 -0700
----------------------------------------------------------------------
.../concurrent/DebuggableThreadPoolExecutor.java | 11 ++++-------
1 file changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8b96bdb/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index 26441ec..46a3216 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -133,7 +133,9 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
public void execute(Runnable command, TraceState state)
{
- super.execute(state == null ? command : new TraceSessionWrapper<Object>(command, state));
+ super.execute(state == null || command instanceof TraceSessionWrapper
+ ? command
+ : new TraceSessionWrapper<Object>(command, state));
}
// execute does not call newTaskFor
@@ -141,7 +143,7 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
public void execute(Runnable command)
{
super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
- ? new TraceSessionWrapper<Object>(command)
+ ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
: command);
}
@@ -261,11 +263,6 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements
{
private final TraceState state;
- public TraceSessionWrapper(Runnable command)
- {
- this(command, null);
- }
-
public TraceSessionWrapper(Callable<T> callable)
{
super(callable);