/** * Attempt to retrieve a leader until one is found */ private void getLeader() { boolean found = getLeaderAttempt(); while (!found) { sleepRandom(); found = getLeaderAttempt(); } }
public synchronized OracleClient getOracleClient() { checkIfClosed(); if (oracleClient == null) { oracleClient = new OracleClient(env); } return oracleClient; }
/** * Return the oracle that the current client is connected to. */ public synchronized String getOracle() { checkClosed(); return currentLeader != null ? currentLeader.getId() : null; }
log.info("The wait command will exit when all notifications are processed"); while (true) { long ts1 = env.getSharedResources().getOracleClient().getStamp().getTxTimestamp(); long ntfyCount = countNotifications(env); long ts2 = env.getSharedResources().getOracleClient().getStamp().getTxTimestamp(); if (ntfyCount == 0 && ts1 == (ts2 - 1)) { log.info("All processing has finished!");
public TransactionImpl(Environment env, Notification trigger) { this(env, trigger, allocateTimestamp(env).getTxTimestamp()); }
private synchronized void close(boolean checkForStaleScan) { if (status != TxStatus.CLOSED) { status = TxStatus.CLOSED; if (checkForStaleScan && !commitAttempted) { Stamp stamp = env.getSharedResources().getOracleClient().getStamp(); if (startTs < stamp.getGcTimestamp()) { throw new StaleScanException(); } } env.getSharedResources().getTimestampTracker().removeTimestamp(startTs); } }
private synchronized void connect() throws IOException, KeeperException, InterruptedException, TTransportException { getLeader(); while (true) { log.debug("Connecting to oracle at " + currentLeader.getId()); String[] hostAndPort = currentLeader.getId().split(":"); String host = hostAndPort[0]; int port = Integer.parseInt(hostAndPort[1]); try { transport = new TFastFramedTransport(new TSocket(host, port)); transport.open(); TProtocol protocol = new TCompactProtocol(transport); client = new OracleService.Client(protocol); log.info("Connected to oracle at " + getOracle()); break; } catch (TTransportException e) { sleepRandom(); getLeader(); } catch (Exception e) { throw new RuntimeException(e); } } }
private Stamp getTimestamp() { return env.getSharedResources().getOracleClient().getStamp(); }
checkClosed(); TimeRequest tr = new TimeRequest(); queue.add(tr); try { long waitTotal = 0; while (!tr.cdl.await(waitPeriod, TimeUnit.SECONDS)) { checkClosed(); waitTotal += waitPeriod; if (waitPeriod < MAX_ORACLE_WAIT_PERIOD) {
@Override public void close() { try { if (oserver != null) { notificationFinder.stop(); mnp.close(); oserver.stop(); env.close(); reporter.close(); if (cluster != null) { cluster.stop(); } } } catch (Exception e) { throw new RuntimeException(e); } }
@Override public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent event) throws Exception { try { if (isConnected() && (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED) || event .getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED))) { synchronized (this) { Participant participant = leaderSelector.getLeader(); if (isLeader(participant) && !leaderSelector.hasLeadership()) { // in case current instance becomes leader, we want to know who came before it. currentLeader = participant; } } } } catch (InterruptedException e) { log.warn("Oracle leadership watcher has been interrupted unexpectedly"); } }
private boolean getLeaderAttempt() { Participant possibleLeader = null; try { possibleLeader = leaderSelector.getLeader(); } catch (KeeperException e) { log.debug("Exception throw in getLeaderAttempt()", e); } catch (Exception e) { throw new RuntimeException(e); } if (isLeader(possibleLeader)) { currentLeader = possibleLeader; return true; } return false; }
@Override public Stamps getTimestamps(String id, int num) throws TException { long start = getTimestampsImpl(id, num); // do this outside of sync stampsHistogram.update(num); return new Stamps(start, gcTsTracker.advertisedGcTimetamp); }
/** * Atomically closes current connection and connects to the current leader */ private synchronized void reconnect() throws InterruptedException, TTransportException, KeeperException, IOException { if (transport.isOpen()) { transport.close(); } connect(); }
private void updateGcTimestamp() throws Exception { List<String> children; try { children = curator.getChildren().forPath(ZookeeperPath.TRANSACTOR_TIMESTAMPS); } catch (NoNodeException nne) { children = Collections.emptyList(); } long oldestTs = Long.MAX_VALUE; boolean nodeFound = false; for (String child : children) { Long ts = LongUtil.fromByteArray(curator.getData().forPath( ZookeeperPath.TRANSACTOR_TIMESTAMPS + "/" + child)); nodeFound = true; if (ts < oldestTs) { oldestTs = ts; } } if (nodeFound) { updateAdvertisedGcTimestamp(oldestTs); } else { updateAdvertisedGcTimestamp(currentTs); } }
private synchronized long getTimestampsImpl(String id, int num) throws TException { if (!started) { throw new IllegalStateException("Received timestamp request but Oracle has not started"); } if (!id.equals(env.getFluoApplicationID())) { throw new IllegalArgumentException("Received timestamp request with a Fluo application ID [" + id + "] that does not match the application ID [" + env.getFluoApplicationID() + "] of the Oracle"); } if (!isLeader) { throw new IllegalStateException("Received timestamp request but Oracle is not leader"); } try { while (num + currentTs >= maxTs) { allocateTimestamp(); } long tmp = currentTs; currentTs += num; return tmp; } catch (Exception e) { throw new TException(e); } }
public OracleClient(Environment env) { this.env = env; responseTimer = MetricsUtil.getTimer(env.getConfiguration(), env.getSharedResources().getMetricRegistry(), env.getMetricNames().getOracleResponseTime()); stampsHistogram = MetricsUtil.getHistogram(env.getConfiguration(), env.getSharedResources() .getMetricRegistry(), env.getMetricNames().getOracleClientStamps()); timestampRetriever = new TimestampRetriever(); thread = new Thread(timestampRetriever); thread.setDaemon(true); thread.start(); }
public TransactionImpl(Environment env) { this(env, null, allocateTimestamp(env).getTxTimestamp()); }
/** * It's possible an Oracle has gone into a bad state. Upon the leader being changed, we want to * update our state */ @Override public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent event) throws Exception { if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { Participant participant = leaderSelector.getLeader(); synchronized (this) { if (isLeader(participant)) { currentLeader = leaderSelector.getLeader(); } else { currentLeader = null; } } } }
/** * Allocate a timestamp */ public Stamp allocateTimestamp() { synchronized (this) { Preconditions.checkState(!closed, "tracker closed "); if (node == null) { Preconditions.checkState(allocationsInProgress == 0, "expected allocationsInProgress == 0 when node == null"); Preconditions.checkState(!updatingZk, "unexpected concurrent ZK update"); createZkNode(getTimestamp().getTxTimestamp()); } allocationsInProgress++; } try { Stamp ts = getTimestamp(); synchronized (this) { timestamps.add(ts.getTxTimestamp()); } return ts; } catch (RuntimeException re) { synchronized (this) { allocationsInProgress--; } throw re; } }