package org.apache.cassandra.db.view;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.CompactionInfo;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.compaction.CompactionInfo.Unit;
import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.sstable.ReducingKeyIterator;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.repair.SystemDistributedKeyspace;
import org.apache.cassandra.service.StorageProxy;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.concurrent.Refs;
public class ViewBuilder extends CompactionInfo.Holder
{
private final ColumnFamilyStore baseCfs;
private final View view;
private final UUID compactionId;
private volatile Token prevToken = null;
private static final Logger logger = LoggerFactory.getLogger(ViewBuilder.class);
private volatile boolean isStopped = false;
public ViewBuilder(ColumnFamilyStore baseCfs, View view)
{
this.baseCfs = baseCfs;
this.view = view;
compactionId = UUIDGen.getTimeUUID();
}
private void buildKey(DecoratedKey key)
{
ReadQuery selectQuery = view.getReadQuery();
if (!selectQuery.selectsKey(key))
{
logger.trace("Skipping {}, view query filters", key);
return;
}
int nowInSec = FBUtilities.nowInSeconds();
SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
try (ReadExecutionController orderGroup = command.executionController();
UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
{
Iterator<Collection<Mutation>> mutations = baseCfs.keyspace.viewManager
.forTable(baseCfs.metadata)
.generateViewUpdates(Collections.singleton(view), data, empty, nowInSec, true);
AtomicLong noBase = new AtomicLong(Long.MAX_VALUE);
mutations.forEachRemaining(m -> StorageProxy.mutateMV(key.getKey(), m, true, noBase, System.nanoTime()));
}
}
public void run()
{
logger.debug("Starting view builder for {}.{}", baseCfs.metadata.ksName, view.name);
logger.trace("Running view builder for {}.{}", baseCfs.metadata.ksName, view.name);
UUID localHostId = SystemKeyspace.getLocalHostId();
String ksname = baseCfs.metadata.ksName, viewName = view.name;
if (SystemKeyspace.isViewBuilt(ksname, viewName))
{
logger.debug("View already marked built for {}.{}", baseCfs.metadata.ksName, view.name);
if (!SystemKeyspace.isViewStatusReplicated(ksname, viewName))
updateDistributed(ksname, viewName, localHostId);
return;
}
Iterable<Range<Token>> ranges = StorageService.instance.getLocalRanges(baseCfs.metadata.ksName);
final Pair<Integer, Token> buildStatus = SystemKeyspace.getViewBuildStatus(ksname, viewName);
Token lastToken;
Function<org.apache.cassandra.db.lifecycle.View, Iterable<SSTableReader>> function;
if (buildStatus == null)
{
logger.debug("Starting new view build. flushing base table {}.{}", baseCfs.metadata.ksName, baseCfs.name);
lastToken = null;
SystemKeyspace.beginViewBuild(ksname, viewName, 0);
}
else
{
lastToken = buildStatus.right;
logger.debug("Resuming view build from token {}. flushing base table {}.{}", lastToken, baseCfs.metadata.ksName, baseCfs.name);
}
baseCfs.forceBlockingFlush();
function = org.apache.cassandra.db.lifecycle.View.selectFunction(SSTableSet.CANONICAL);
prevToken = lastToken;
long keysBuilt = 0;
try (Refs<SSTableReader> sstables = baseCfs.selectAndReference(function).refs;
ReducingKeyIterator iter = new ReducingKeyIterator(sstables))
{
SystemDistributedKeyspace.startViewBuild(ksname, viewName, localHostId);
while (!isStopped && iter.hasNext())
{
DecoratedKey key = iter.next();
Token token = key.getToken();
if (lastToken == null || lastToken.compareTo(token) < 0)
{
for (Range<Token> range : ranges)
{
if (range.contains(token))
{
buildKey(key);
++keysBuilt;
if (prevToken == null || prevToken.compareTo(token) != 0)
{
SystemKeyspace.updateViewBuildStatus(ksname, viewName, key.getToken());
prevToken = token;
}
}
}
lastToken = null;
}
}
if (!isStopped)
{
logger.debug("Marking view({}.{}) as built covered {} keys ", ksname, viewName, keysBuilt);
SystemKeyspace.finishViewBuildStatus(ksname, viewName);
updateDistributed(ksname, viewName, localHostId);
}
else
{
logger.debug("Stopped build for view({}.{}) after covering {} keys", ksname, viewName, keysBuilt);
}
}
catch (Exception e)
{
ScheduledExecutors.nonPeriodicTasks.schedule(() -> CompactionManager.instance.submitViewBuilder(this),
5,
TimeUnit.MINUTES);
logger.warn("Materialized View failed to complete, sleeping 5 minutes before restarting", e);
}
}
private void updateDistributed(String ksname, String viewName, UUID localHostId)
{
try
{
SystemDistributedKeyspace.successfulViewBuild(ksname, viewName, localHostId);
SystemKeyspace.setViewBuiltReplicated(ksname, viewName);
}
catch (Exception e)
{
ScheduledExecutors.nonPeriodicTasks.schedule(() -> CompactionManager.instance.submitViewBuilder(this),
5,
TimeUnit.MINUTES);
logger.warn("Failed to updated the distributed status of view, sleeping 5 minutes before retrying", e);
}
}
public CompactionInfo getCompactionInfo()
{
long rangesCompleted = 0, rangesTotal = 0;
Token lastToken = prevToken;
for (Range<Token> range : StorageService.instance.getLocalRanges(baseCfs.keyspace.getName()))
{
rangesTotal++;
if ((lastToken != null) && lastToken.compareTo(range.right) > 0)
rangesCompleted++;
}
return new CompactionInfo(baseCfs.metadata, OperationType.VIEW_BUILD, rangesCompleted, rangesTotal, Unit.RANGES, compactionId);
}
public void stop()
{
isStopped = true;
}
}