@Override public boolean isActive(TInfo tinfo, long tid) { return transactionWatcher.isActive(tid); }
protected Arbitrator getArbitrator(ServerContext context) { return new ZooArbitrator(context); } }
public <T> T run(String ztxBulk, long tid, Callable<T> callable) throws Exception { synchronized (counts) { if (!arbitrator.transactionAlive(ztxBulk, tid)) { throw new Exception("Transaction " + tid + " of type " + ztxBulk + " is no longer active"); } increment(tid); } try { return callable.call(); } finally { decrement(tid); } }
private void initiateSort(String sortId, String source, final String destination) throws KeeperException, InterruptedException { String work = source + "|" + destination; new DistributedWorkQueue(master.getZooKeeperRoot() + Constants.ZRECOVERY, master.getConfiguration()).addWork(sortId, work.getBytes(UTF_8)); synchronized (this) { sortsQueued.add(sortId); } final String path = master.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId; log.info("Created zookeeper entry {} with data {}", path, work); }
public RecoveryManager(Master master) { this.master = master; executor = Executors.newScheduledThreadPool(4, new NamingThreadFactory("Walog sort starter ")); zooCache = new ZooCache(master.getContext().getZooReaderWriter(), null); try { List<String> workIDs = new DistributedWorkQueue( master.getZooKeeperRoot() + Constants.ZRECOVERY, master.getConfiguration()) .getWorkQueued(); sortsQueued.addAll(workIDs); } catch (Exception e) { log.warn("{}", e.getMessage(), e); } }
public void startWatchingForRecoveryLogs(ThreadPoolExecutor distWorkQThreadPool) throws KeeperException, InterruptedException { this.threadPool = distWorkQThreadPool; new DistributedWorkQueue(context.getZooKeeperRoot() + Constants.ZRECOVERY, conf) .startProcessing(new LogProcessor(), this.threadPool); }
/** * Adds work to the queue, automatically converting the String to bytes using UTF-8 */ public void addWork(String workId, String data) throws KeeperException, InterruptedException { addWork(workId, data.getBytes(UTF_8)); }
/** * Initialize the DistributedWorkQueue using the proper ZK location */ protected void initializeWorkQueue(AccumuloConfiguration conf) { workQueue = new DistributedWorkQueue( ZooUtil.getRoot(client.getInstanceID()) + ReplicationConstants.ZOO_WORK_QUEUE, conf); }
try { try { processor.newProcessor().process(child, zoo.getData(childPath, null));
@Override public Repo<Master> call(long tid, Master master) throws Exception { ZooArbitrator.stop(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid); return new CleanUpBulkImport(info); } }
@Override public void undo(long tid, Master environment) throws Exception { // unreserve sourceDir/error directories Utils.unreserveHdfsDirectory(environment, bulkInfo.sourceDir, tid); Utils.getReadLock(environment, bulkInfo.tableId, tid).unlock(); TransactionWatcher.ZooArbitrator.cleanup(environment.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid); } }
lookForWork(processor, children);
@Override public void run() { // gather the list of transactions the tablets have cached final Set<Long> tids = new HashSet<>(); for (Tablet tablet : server.getOnlineTablets()) { tids.addAll(tablet.getBulkIngestedFiles().keySet()); } try { // get the current transactions from ZooKeeper final Set<Long> allTransactionsAlive = ZooArbitrator.allTransactionsAlive(server.getContext(), Constants.BULK_ARBITRATOR_TYPE); // remove any that are still alive tids.removeAll(allTransactionsAlive); // cleanup any memory of these transactions for (Tablet tablet : server.getOnlineTablets()) { tablet.cleanupBulkLoadedFiles(tids); } } catch (KeeperException | InterruptedException e) { // we'll just clean it up again later log.debug("Error reading bulk import live transactions {}", e); } }
@Override public void assignWork() { if (workQueue == null) { initializeWorkQueue(conf); } initializeQueuedWork(); if (zooCache == null) { zooCache = new ZooCache(workQueue.getZooReaderWriter()); } // Get the maximum number of entries we want to queue work for (or the default) this.maxQueueSize = conf.getCount(Property.REPLICATION_MAX_WORK_QUEUE); // Scan over the work records, adding the work to the queue createWork(); // Keep the state of the work we queued correct cleanupFinishedWork(); }
/** * Distribute the work for the given path with filename * * @param path * Path to the file being replicated * @param target * Target for the file to be replicated to */ @Override protected boolean queueWork(Path path, ReplicationTarget target) { String queueKey = DistributedWorkQueueWorkAssignerHelper.getQueueKey(path.getName(), target); if (queuedWork.contains(queueKey)) { log.debug("{} is already queued to be replicated to {}, not re-queueing", path, target); return false; } try { log.debug("Queued work for {} and {}", queueKey, path); workQueue.addWork(queueKey, path.toString()); queuedWork.add(queueKey); } catch (KeeperException | InterruptedException e) { log.warn("Could not queue work for {}", path, e); return false; } return true; }
/** * Run task only if transaction is still active in zookeeper. If the tx is no longer active then * that task is not run and a debug message is logged indicating the task was ignored. */ public void runQuietly(String ztxBulk, long tid, Runnable task) { synchronized (counts) { try { if (!arbitrator.transactionAlive(ztxBulk, tid)) { log.debug("Transaction " + tid + " of type " + ztxBulk + " is no longer active."); return; } } catch (Exception e) { log.warn("Unable to check if transaction " + tid + " of type " + ztxBulk + " is alive ", e); return; } increment(tid); } try { task.run(); } finally { decrement(tid); } }
public TransactionWatcher(ServerContext context) { this(new ZooArbitrator(context)); }
@Override public Repo<Master> call(long tid, Master master) throws Exception { ZooArbitrator.stop(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid); return new CopyFailed(tableId, source, bulk, error); } }
@Override public void undo(long tid, Master environment) throws Exception { // unreserve source/error directories Utils.unreserveHdfsDirectory(environment, sourceDir, tid); Utils.unreserveHdfsDirectory(environment, errorDir, tid); Utils.getReadLock(environment, tableId, tid).unlock(); ZooArbitrator.cleanup(environment.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid); } }
protected Arbitrator getArbitrator(ServerContext context) { Objects.nonNull(context); return new ZooArbitrator(context); }