> replicaCursors =
new ConcurrentSkipListMap<>();
private ReplicationEnvironment replicationEnv;
private final ReplicationServerCfg config;
private final File dbDirectory;
/**
* The handler of the changelog database, the database stores the relation
* between a change number and the associated cookie.
*
* @GuardedBy("cnIndexDBLock")
*/
private FileChangeNumberIndexDB cnIndexDB;
private final AtomicReference cnIndexer = new AtomicReference<>();
/** Used for protecting {@link ChangeNumberIndexDB} related state. */
private final Object cnIndexDBLock = new Object();
/**
* The purge delay (in milliseconds). Records in the changelog DB that are
* older than this delay might be removed.
*/
private volatile long purgeDelayInMillis;
private final AtomicReference cnPurger = new AtomicReference<>();
/** The local replication server. */
private final ReplicationServer replicationServer;
private final AtomicBoolean shutdown = new AtomicBoolean();
private static final RepositionableCursor EMPTY_CURSOR = Log.getEmptyCursor();
private static final DBCursor EMPTY_CURSOR_REPLICA_DB =
new FileReplicaDBCursor(EMPTY_CURSOR, null, AFTER_MATCHING_KEY);
/**
* Creates a new changelog DB.
*
* @param replicationServer
* the local replication server.
* @param config
* the replication server configuration
* @throws ConfigException
* if a problem occurs opening the supplied directory
*/
public FileChangelogDB(final ReplicationServer replicationServer, final ReplicationServerCfg config)
throws ConfigException
{
this.config = config;
this.replicationServer = replicationServer;
this.dbDirectory = makeDir(config.getReplicationDBDirectory());
}
private File makeDir(final String dbDirName) throws ConfigException
{
// Check that this path exists or create it.
final File dbDirectory = getFileForPath(dbDirName);
try
{
if (!dbDirectory.exists())
{
dbDirectory.mkdir();
}
return dbDirectory;
}
catch (Exception e)
{
final LocalizableMessageBuilder mb = new LocalizableMessageBuilder(
e.getLocalizedMessage()).append(" ").append(String.valueOf(dbDirectory));
throw new ConfigException(ERR_FILE_CHECK_CREATE_FAILED.get(mb.toString()), e);
}
}
private Map getDomainMap(final DN baseDN)
{
final Map domainMap = domainToReplicaDBs.get(baseDN);
if (domainMap != null)
{
return domainMap;
}
return Collections.emptyMap();
}
private FileReplicaDB getReplicaDB(final DN baseDN, final int serverId)
{
return getDomainMap(baseDN).get(serverId);
}
/**
* Returns a {@link FileReplicaDB}, possibly creating it.
*
* @param baseDN
* the baseDN for which to create a ReplicaDB
* @param serverId
* the serverId for which to create a ReplicaDB
* @param server
* the ReplicationServer
* @return a Pair with the FileReplicaDB and a boolean indicating whether it has been created
* @throws ChangelogException
* if a problem occurred with the database
*/
Pair getOrCreateReplicaDB(final DN baseDN, final int serverId,
final ReplicationServer server) throws ChangelogException
{
while (!shutdown.get())
{
final ConcurrentMap domainMap = getExistingOrNewDomainMap(baseDN);
final Pair result = getExistingOrNewReplicaDB(domainMap, serverId, baseDN, server);
if (result != null)
{
final Boolean dbWasCreated = result.getSecond();
if (dbWasCreated)
{ // new replicaDB => update all cursors with it
final List cursors = registeredDomainCursors.get(baseDN);
if (cursors != null && !cursors.isEmpty())
{
for (DomainDBCursor cursor : cursors)
{
cursor.addReplicaDB(serverId, null);
}
}
}
return result;
}
}
throw new ChangelogException(ERR_CANNOT_CREATE_REPLICA_DB_BECAUSE_CHANGELOG_DB_SHUTDOWN.get());
}
private ConcurrentMap getExistingOrNewDomainMap(final DN baseDN)
{
// happy path: the domainMap already exists
final ConcurrentMap currentValue = domainToReplicaDBs.get(baseDN);
if (currentValue != null)
{
return currentValue;
}
// unlucky, the domainMap does not exist: take the hit and create the
// newValue, even though the same could be done concurrently by another thread
final ConcurrentMap newValue = new ConcurrentHashMap<>();
final ConcurrentMap previousValue = domainToReplicaDBs.putIfAbsent(baseDN, newValue);
if (previousValue != null)
{
// there was already a value associated to the key, let's use it
return previousValue;
}
// we just created a new domain => update all cursors
for (MultiDomainDBCursor cursor : registeredMultiDomainCursors)
{
cursor.addDomain(baseDN, null);
}
return newValue;
}
private Pair getExistingOrNewReplicaDB(final ConcurrentMap domainMap,
final int serverId, final DN baseDN, final ReplicationServer server) throws ChangelogException
{
// happy path: the replicaDB already exists
FileReplicaDB currentValue = domainMap.get(serverId);
if (currentValue != null)
{
return Pair.of(currentValue, false);
}
// unlucky, the replicaDB does not exist: take the hit and synchronize
// on the domainMap to create a new ReplicaDB
synchronized (domainMap)
{
// double-check
currentValue = domainMap.get(serverId);
if (currentValue != null)
{
return Pair.of(currentValue, false);
}
if (domainToReplicaDBs.get(baseDN) != domainMap)
{
// The domainMap could have been concurrently removed because
// 1) a shutdown was initiated or 2) an initialize was called.
// Return will allow the code to:
// 1) shutdown properly or 2) lazily recreate the replicaDB
return null;
}
final FileReplicaDB newDB = new FileReplicaDB(serverId, baseDN, server, replicationEnv);
domainMap.put(serverId, newDB);
return Pair.of(newDB, true);
}
}
/** {@inheritDoc} */
@Override
public void initializeDB()
{
try
{
final File dbDir = getFileForPath(config.getReplicationDBDirectory());
replicationEnv = new ReplicationEnvironment(dbDir.getAbsolutePath(), replicationServer, TimeService.SYSTEM);
final ChangelogState changelogState = replicationEnv.getChangelogState();
initializeToChangelogState(changelogState);
if (config.isComputeChangeNumber())
{
startIndexer();
}
setPurgeDelay(replicationServer.getPurgeDelay());
}
catch (ChangelogException e)
{
logger.traceException(e);
logger.error(ERR_COULD_NOT_READ_DB, this.dbDirectory.getAbsolutePath(), e.getLocalizedMessage());
}
}
private void initializeToChangelogState(final ChangelogState changelogState)
throws ChangelogException
{
for (Map.Entry entry : changelogState.getDomainToGenerationId().entrySet())
{
replicationServer.getReplicationServerDomain(entry.getKey(), true).initGenerationID(entry.getValue());
}
for (Map.Entry> entry : changelogState.getDomainToServerIds().entrySet())
{
for (int serverId : entry.getValue())
{
getOrCreateReplicaDB(entry.getKey(), serverId, replicationServer);
}
}
}
private void shutdownChangeNumberIndexDB() throws ChangelogException
{
synchronized (cnIndexDBLock)
{
if (cnIndexDB != null)
{
cnIndexDB.shutdown();
}
}
}
/** {@inheritDoc} */
@Override
public void shutdownDB() throws ChangelogException
{
if (!this.shutdown.compareAndSet(false, true))
{ // shutdown has already been initiated
return;
}
shutdownCNIndexerAndPurger();
// Remember the first exception because :
// - we want to try to remove everything we want to remove
// - then throw the first encountered exception
ChangelogException firstException = null;
// now we can safely shutdown all DBs
try
{
shutdownChangeNumberIndexDB();
}
catch (ChangelogException e)
{
firstException = e;
}
for (Iterator> it =
this.domainToReplicaDBs.values().iterator(); it.hasNext();)
{
final ConcurrentMap domainMap = it.next();
synchronized (domainMap)
{
it.remove();
for (FileReplicaDB replicaDB : domainMap.values())
{
replicaDB.shutdown();
}
}
}
if (replicationEnv != null)
{
replicationEnv.shutdown();
}
if (firstException != null)
{
throw firstException;
}
}
private void shutdownCNIndexerAndPurger()
{
final ChangeNumberIndexer indexer = cnIndexer.getAndSet(null);
if (indexer != null)
{
indexer.initiateShutdown();
}
final ChangelogDBPurger purger = cnPurger.getAndSet(null);
if (purger != null)
{
purger.initiateShutdown();
}
// wait for shutdown of the threads holding cursors
try
{
if (indexer != null)
{
indexer.join();
}
if (purger != null)
{
purger.join();
}
}
catch (InterruptedException e)
{
// do nothing: we are already shutting down
}
}
/**
* Clears all records from the changelog (does not remove the changelog itself).
*
* @throws ChangelogException
* If an error occurs when clearing the changelog.
*/
public void clearDB() throws ChangelogException
{
if (!dbDirectory.exists())
{
return;
}
// Remember the first exception because :
// - we want to try to remove everything we want to remove
// - then throw the first encountered exception
ChangelogException firstException = null;
for (DN baseDN : this.domainToReplicaDBs.keySet())
{
removeDomain(baseDN);
}
synchronized (cnIndexDBLock)
{
if (cnIndexDB != null)
{
try
{
cnIndexDB.clear();
}
catch (ChangelogException e)
{
firstException = e;
}
try
{
shutdownChangeNumberIndexDB();
}
catch (ChangelogException e)
{
if (firstException == null)
{
firstException = e;
}
else
{
logger.traceException(e);
}
}
cnIndexDB = null;
}
}
if (firstException != null)
{
throw firstException;
}
}
/** {@inheritDoc} */
@Override
public void removeDB() throws ChangelogException
{
shutdownDB();
StaticUtils.recursiveDelete(dbDirectory);
}
/** {@inheritDoc} */
@Override
public ServerState getDomainOldestCSNs(DN baseDN)
{
final ServerState result = new ServerState();
for (FileReplicaDB replicaDB : getDomainMap(baseDN).values())
{
result.update(replicaDB.getOldestCSN());
}
return result;
}
/** {@inheritDoc} */
@Override
public ServerState getDomainNewestCSNs(DN baseDN)
{
final ServerState result = new ServerState();
for (FileReplicaDB replicaDB : getDomainMap(baseDN).values())
{
result.update(replicaDB.getNewestCSN());
}
return result;
}
/** {@inheritDoc} */
@Override
public void removeDomain(DN baseDN) throws ChangelogException
{
// Remember the first exception because :
// - we want to try to remove everything we want to remove
// - then throw the first encountered exception
ChangelogException firstException = null;
// 1- clear the replica DBs
Map domainMap = domainToReplicaDBs.get(baseDN);
if (domainMap != null)
{
final ChangeNumberIndexer indexer = this.cnIndexer.get();
if (indexer != null)
{
indexer.clear(baseDN);
}
synchronized (domainMap)
{
domainMap = domainToReplicaDBs.remove(baseDN);
for (FileReplicaDB replicaDB : domainMap.values())
{
try
{
replicaDB.clear();
}
catch (ChangelogException e)
{
firstException = e;
}
replicaDB.shutdown();
}
}
}
// 2- clear the changelogstate DB
try
{
replicationEnv.clearGenerationId(baseDN);
}
catch (ChangelogException e)
{
if (firstException == null)
{
firstException = e;
}
else
{
logger.traceException(e);
}
}
if (firstException != null)
{
throw firstException;
}
}
/** {@inheritDoc} */
@Override
public void setPurgeDelay(final long purgeDelayInMillis)
{
this.purgeDelayInMillis = purgeDelayInMillis;
// Rotation time interval for CN Index DB log file
// needs to be a fraction of the purge delay
// to ensure there is at least one file to purge
replicationEnv.setCNIndexDBRotationInterval(purgeDelayInMillis / 2);
if (purgeDelayInMillis > 0)
{
startCNPurger();
}
else
{
final ChangelogDBPurger purgerToStop = cnPurger.getAndSet(null);
if (purgerToStop != null)
{ // stop this purger
purgerToStop.initiateShutdown();
}
}
}
private void startCNPurger()
{
final ChangelogDBPurger newPurger = new ChangelogDBPurger();
if (cnPurger.compareAndSet(null, newPurger))
{ // no purger was running, run this new one
newPurger.start();
}
else
{ // a purger was already running, just wake that one up
// to verify if some entries can be purged
final ChangelogDBPurger currentPurger = cnPurger.get();
synchronized (currentPurger)
{
currentPurger.notify();
}
}
}
/** {@inheritDoc} */
@Override
public void setComputeChangeNumber(final boolean computeChangeNumber)
throws ChangelogException
{
if (computeChangeNumber)
{
startIndexer();
}
else
{
final ChangeNumberIndexer indexer = cnIndexer.getAndSet(null);
if (indexer != null)
{
indexer.initiateShutdown();
}
}
}
void resetChangeNumberIndex(long newFirstCN, DN baseDN, CSN newFirstCSN) throws ChangelogException
{
if (!config.isComputeChangeNumber())
{
throw new ChangelogException(ERR_REPLICATION_CHANGE_NUMBER_DISABLED.get(baseDN));
}
if (!getDomainNewestCSNs(baseDN).cover(newFirstCSN))
{
throw new ChangelogException(ERR_CHANGELOG_RESET_CHANGE_NUMBER_CHANGE_NOT_PRESENT.get(newFirstCN, baseDN,
newFirstCSN));
}
if (getDomainOldestCSNs(baseDN).getCSN(newFirstCSN.getServerId()).isNewerThan(newFirstCSN))
{
throw new ChangelogException(ERR_CHANGELOG_RESET_CHANGE_NUMBER_CSN_TOO_OLD.get(newFirstCN, newFirstCSN));
}
shutdownCNIndexerAndPurger();
synchronized (cnIndexDBLock)
{
cnIndexDB.clearAndSetChangeNumber(newFirstCN);
cnIndexDB.addRecord(new ChangeNumberIndexRecord(newFirstCN, baseDN, newFirstCSN));
}
startIndexer();
if (purgeDelayInMillis > 0)
{
startCNPurger();
}
}
private void startIndexer()
{
final ChangeNumberIndexer indexer = new ChangeNumberIndexer(this, replicationEnv);
if (cnIndexer.compareAndSet(null, indexer))
{
indexer.start();
}
}
@Override
public ChangeNumberIndexDB getChangeNumberIndexDB()
{
synchronized (cnIndexDBLock)
{
if (cnIndexDB == null)
{
try
{
cnIndexDB = new FileChangeNumberIndexDB(this, replicationEnv);
}
catch (Exception e)
{
logger.traceException(e);
logger.error(ERR_CHANGENUMBER_DATABASE, e.getLocalizedMessage());
}
}
return cnIndexDB;
}
}
/** {@inheritDoc} */
@Override
public ReplicationDomainDB getReplicationDomainDB()
{
return this;
}
/** {@inheritDoc} */
@Override
public MultiDomainDBCursor getCursorFrom(final MultiDomainServerState startState, CursorOptions options)
throws ChangelogException
{
final Set excludedDomainDns = Collections.emptySet();
return getCursorFrom(startState, options, excludedDomainDns);
}
/** {@inheritDoc} */
@Override
public MultiDomainDBCursor getCursorFrom(final MultiDomainServerState startState,
CursorOptions options, final Set excludedDomainDns) throws ChangelogException
{
final MultiDomainDBCursor cursor = new MultiDomainDBCursor(this, options);
registeredMultiDomainCursors.add(cursor);
for (DN baseDN : domainToReplicaDBs.keySet())
{
if (!excludedDomainDns.contains(baseDN))
{
cursor.addDomain(baseDN, startState.getServerState(baseDN));
}
}
return cursor;
}
/** {@inheritDoc} */
@Override
public DBCursor getCursorFrom(final DN baseDN, final ServerState startState, CursorOptions options)
throws ChangelogException
{
final DomainDBCursor cursor = newDomainDBCursor(baseDN, options);
for (int serverId : getDomainMap(baseDN).keySet())
{
// get the last already sent CSN from that server to get a cursor
final CSN lastCSN = startState != null ? startState.getCSN(serverId) : null;
cursor.addReplicaDB(serverId, lastCSN);
}
return cursor;
}
private DomainDBCursor newDomainDBCursor(final DN baseDN, final CursorOptions options)
{
final DomainDBCursor cursor = new DomainDBCursor(baseDN, this, options);
putCursor(registeredDomainCursors, baseDN, cursor);
return cursor;
}
private CSN getOfflineCSN(DN baseDN, int serverId, CSN startAfterCSN)
{
final MultiDomainServerState offlineReplicas =
replicationEnv.getChangelogState().getOfflineReplicas();
final CSN offlineCSN = offlineReplicas.getCSN(baseDN, serverId);
if (offlineCSN != null
&& (startAfterCSN == null || startAfterCSN.isOlderThan(offlineCSN)))
{
return offlineCSN;
}
return null;
}
@Override
public DBCursor getCursorFrom(final DN baseDN, final int serverId, final CSN startCSN,
CursorOptions options) throws ChangelogException
{
final FileReplicaDB replicaDB = getReplicaDB(baseDN, serverId);
if (replicaDB != null)
{
final CSN actualStartCSN = startCSN != null ? startCSN : options.getDefaultCSN();
final DBCursor cursor = replicaDB.generateCursorFrom(
actualStartCSN, options.getKeyMatchingStrategy(), options.getPositionStrategy());
final CSN offlineCSN = getOfflineCSN(baseDN, serverId, actualStartCSN);
final ReplicaId replicaId = ReplicaId.of(baseDN, serverId);
final ReplicaCursor replicaCursor = new ReplicaCursor(cursor, offlineCSN, replicaId, this);
putCursor(replicaCursors, replicaId, replicaCursor);
return replicaCursor;
}
return EMPTY_CURSOR_REPLICA_DB;
}
private void putCursor(ConcurrentSkipListMap> map, final K key, final V cursor)
{
CopyOnWriteArrayList cursors = map.get(key);
if (cursors == null)
{
cursors = new CopyOnWriteArrayList<>();
CopyOnWriteArrayList previousValue = map.putIfAbsent(key, cursors);
if (previousValue != null)
{
cursors = previousValue;
}
}
cursors.add(cursor);
}
/** {@inheritDoc} */
@Override
public void unregisterCursor(final DBCursor> cursor)
{
if (cursor instanceof MultiDomainDBCursor)
{
registeredMultiDomainCursors.remove(cursor);
}
else if (cursor instanceof DomainDBCursor)
{
final DomainDBCursor domainCursor = (DomainDBCursor) cursor;
final List cursors = registeredDomainCursors.get(domainCursor.getBaseDN());
if (cursors != null)
{
cursors.remove(cursor);
}
}
else if (cursor instanceof ReplicaCursor)
{
final ReplicaCursor replicaCursor = (ReplicaCursor) cursor;
final List cursors = replicaCursors.get(replicaCursor.getReplicaId());
if (cursors != null)
{
cursors.remove(cursor);
}
}
}
/** {@inheritDoc} */
@Override
public boolean publishUpdateMsg(final DN baseDN, final UpdateMsg updateMsg) throws ChangelogException
{
final CSN csn = updateMsg.getCSN();
final Pair pair = getOrCreateReplicaDB(baseDN,
csn.getServerId(), replicationServer);
final FileReplicaDB replicaDB = pair.getFirst();
replicaDB.add(updateMsg);
ChangelogBackend.getInstance().notifyCookieEntryAdded(baseDN, updateMsg);
final ChangeNumberIndexer indexer = cnIndexer.get();
if (indexer != null)
{
notifyReplicaOnline(indexer, baseDN, csn.getServerId());
indexer.publishUpdateMsg(baseDN, updateMsg);
}
return pair.getSecond(); // replica DB was created
}
/** {@inheritDoc} */
@Override
public void replicaHeartbeat(final DN baseDN, final CSN heartbeatCSN) throws ChangelogException
{
final ChangeNumberIndexer indexer = cnIndexer.get();
if (indexer != null)
{
notifyReplicaOnline(indexer, baseDN, heartbeatCSN.getServerId());
indexer.publishHeartbeat(baseDN, heartbeatCSN);
}
}
private void notifyReplicaOnline(final ChangeNumberIndexer indexer, final DN baseDN, final int serverId)
throws ChangelogException
{
if (indexer.isReplicaOffline(baseDN, serverId))
{
replicationEnv.notifyReplicaOnline(baseDN, serverId);
}
updateCursorsWithOfflineCSN(baseDN, serverId, null);
}
/** {@inheritDoc} */
@Override
public void notifyReplicaOffline(final DN baseDN, final CSN offlineCSN) throws ChangelogException
{
replicationEnv.notifyReplicaOffline(baseDN, offlineCSN);
final ChangeNumberIndexer indexer = cnIndexer.get();
if (indexer != null)
{
indexer.replicaOffline(baseDN, offlineCSN);
}
updateCursorsWithOfflineCSN(baseDN, offlineCSN.getServerId(), offlineCSN);
}
private void updateCursorsWithOfflineCSN(final DN baseDN, final int serverId, final CSN offlineCSN)
{
final List cursors = replicaCursors.get(ReplicaId.of(baseDN, serverId));
if (cursors != null)
{
for (ReplicaCursor cursor : cursors)
{
cursor.setOfflineCSN(offlineCSN);
}
}
}
/**
* The thread purging the changelogDB on a regular interval. Records are
* purged from the changelogDB if they are older than a delay specified in
* seconds. The purge process works in two steps:
*
* - first purge the changeNumberIndexDB and retrieve information to drive
* replicaDBs purging
* - proceed to purge each replicaDBs based on the information collected
* when purging the changeNumberIndexDB
*
*/
private final class ChangelogDBPurger extends DirectoryThread
{
private static final int DEFAULT_SLEEP = 500;
protected ChangelogDBPurger()
{
super("changelog DB purger");
}
/** {@inheritDoc} */
@Override
public void run()
{
// initialize CNIndexDB
getChangeNumberIndexDB();
while (!isShutdownInitiated())
{
try
{
final long purgeTimestamp = TimeThread.getTime() - purgeDelayInMillis;
final CSN purgeCSN = new CSN(purgeTimestamp, 0, 0);
final CSN oldestNotPurgedCSN;
// next code assumes that the compute-change-number config
// never changes during the life time of an RS
if (!config.isComputeChangeNumber())
{
oldestNotPurgedCSN = purgeCSN;
}
else
{
final FileChangeNumberIndexDB localCNIndexDB = cnIndexDB;
if (localCNIndexDB == null)
{ // shutdown has been initiated
return;
}
oldestNotPurgedCSN = localCNIndexDB.purgeUpTo(purgeCSN);
if (oldestNotPurgedCSN == null)
{ // shutdown may have been initiated...
// ... or the change number index DB is empty,
// wait for new changes to come in.
// Note we cannot sleep for as long as the purge delay
// (3 days default), because we might receive late updates
// that will have to be purged before the purge delay elapses.
// This can particularly happen in case of network partitions.
if (!isShutdownInitiated())
{
synchronized (this)
{
if (!isShutdownInitiated())
{
wait(DEFAULT_SLEEP);
}
}
}
continue;
}
}
for (final Map domainMap : domainToReplicaDBs.values())
{
for (final FileReplicaDB replicaDB : domainMap.values())
{
replicaDB.purgeUpTo(oldestNotPurgedCSN);
}
}
if (!isShutdownInitiated())
{
synchronized (this)
{
if (!isShutdownInitiated())
{
wait(computeSleepTimeUntilNextPurge(oldestNotPurgedCSN));
}
}
}
}
catch (InterruptedException e)
{
// shutdown initiated?
}
catch (Exception e)
{
logger.error(ERR_EXCEPTION_CHANGELOG_TRIM_FLUSH, stackTraceToSingleLineString(e));
if (replicationServer != null)
{
replicationServer.shutdown();
}
}
}
}
private long computeSleepTimeUntilNextPurge(CSN notPurgedCSN)
{
final long nextPurgeTime = notPurgedCSN.getTime();
final long currentPurgeTime = TimeThread.getTime() - purgeDelayInMillis;
if (currentPurgeTime <= nextPurgeTime)
{
// sleep till the next CSN to purge,
return nextPurgeTime - currentPurgeTime;
}
// wait a bit before purging more
return DEFAULT_SLEEP;
}
/** {@inheritDoc} */
@Override
public void initiateShutdown()
{
super.initiateShutdown();
synchronized (this)
{
notify(); // wake up the purger thread for faster shutdown
}
}
}
}