Skip to content
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public CassandraStreamReader(StreamMessageHeader header, CassandraStreamHeader s
{
// we should only ever be streaming pending repair
// sstables if the session has a pending repair id
assert session.getPendingRepair().equals(header.pendingRepair);
assert session.getPendingRepair().equals(header.pendingRepair) : session.getPendingRepair() + " != " + header.pendingRepair;
}
this.session = session;
this.tableId = header.tableId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -223,9 +223,13 @@ else if (message.verb() == STATUS_RSP)
}
catch (Exception e)
{
logger.error("Got error processing {}, removing parent repair session", message.verb());
if (desc != null && desc.parentSessionId != null)
{
logger.error("Got error processing {}, removing parent repair session {}", message.verb(), desc.parentSessionId);
ActiveRepairService.instance.removeParentRepairSession(desc.parentSessionId);
}
else
logger.error("Got error processing {}, removing parent repair session", message.verb());
throw new RuntimeException(e);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.commons.lang3.time.DurationFormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -274,7 +275,7 @@ public synchronized void fail()
logger.error("Can't transition endpoints {} to FAILED", cantFail, new RuntimeException());
return;
}
logger.info("Incremental repair session {} failed", sessionID);
logger.info("Incremental repair session {} failed \n {}", sessionID, ExceptionUtils.getStackTrace(new Exception()));
sendFailureMessageToParticipants();
setAll(State.FAILED);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;

import org.apache.commons.lang3.exception.ExceptionUtils;

import org.apache.cassandra.cql3.PageSize;
import org.apache.cassandra.db.compaction.AbstractCompactionTask;
import org.apache.cassandra.db.compaction.CleanupTask;
Expand Down Expand Up @@ -417,7 +419,7 @@ public boolean canCleanup(UUID sessionID)
*/
public void cancelSession(UUID sessionID, boolean force)
{
logger.info("Cancelling local repair session {}", sessionID);
logger.info("Cancelling local repair session {} \n {}", sessionID, ExceptionUtils.getStackTrace(new Exception()));
LocalSession session = getSession(sessionID);
Preconditions.checkArgument(session != null, "Session {} does not exist", sessionID);
Preconditions.checkArgument(force || session.coordinator.equals(getBroadcastAddressAndPort()),
Expand Down Expand Up @@ -791,8 +793,8 @@ void setStateAndSave(LocalSession session, ConsistentSession.State state)
synchronized (session)
{
Preconditions.checkArgument(session.getState().canTransitionTo(state),
"Invalid state transition %s -> %s",
session.getState(), state);
"Invalid state transition %s -> %s for session %s",
session.getState(), state, session.sessionID);
logger.trace("Changing LocalSession state from {} -> {} for {}", session.getState(), state, session.sessionID);
boolean wasCompleted = session.isCompleted();
session.setState(state);
Expand Down Expand Up @@ -831,7 +833,7 @@ public void failSession(LocalSession session, boolean sendMessage)
}
else if (session.getState() != FAILED)
{
logger.info("Failing local repair session {}", session.sessionID);
logger.info("Failing local repair session {} \n {}", session.sessionID, ExceptionUtils.getStackTrace(new Exception()));
setStateAndSave(session, FAILED);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;

import org.apache.commons.lang3.exception.ExceptionUtils;

import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.locator.EndpointsByRange;
Expand Down Expand Up @@ -413,6 +415,7 @@ public void run()

public synchronized void terminateSessions()
{
logger.info("Terminating repair sessions \n {}", ExceptionUtils.getStackTrace(new Exception()));
Throwable cause = new IOException("Terminate session is called");
for (RepairSession session : sessions.values())
{
Expand Down