void readConnectResult() throws IOException {
    if (LOG.isTraceEnabled()) {
      StringBuilder buf = new StringBuilder("0x[");
      for (byte b : incomingBuffer.array()) {
        buf.append(Integer.toHexString(b) + ",");
      }
      buf.append("]");
      LOG.trace("readConnectResult " + incomingBuffer.remaining() + " " + buf.toString());
    }
    ByteBufferInputStream bbis = new ByteBufferInputStream(incomingBuffer);
    BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
    ConnectResponse conRsp = new ConnectResponse();
    conRsp.deserialize(bbia, "connect");

    // read "is read-only" flag
    boolean isRO = false;
    try {
      isRO = bbia.readBool("readOnly");
    } catch (IOException e) {
      // this is ok -- just a packet from an old server which
      // doesn't contain readOnly field
      LOG.warn("Connected to an old server; r-o mode will be unavailable");
    }

    this.sessionId = conRsp.getSessionId();
    sendThread.onConnected(conRsp.getTimeOut(), this.sessionId, conRsp.getPasswd(), isRO);
  }
 /**
  * apply the log files to the datatree
  *
  * @param oldTree the datatreee to apply the logs to
  * @param logFiles the logs to be applied
  * @throws IOException
  */
 private long processLogFiles(DataTreeV1 oldTree, File[] logFiles) throws IOException {
   long zxid = 0;
   for (File f : logFiles) {
     LOG.info("Processing log file: " + f);
     InputStream logIs = new BufferedInputStream(new FileInputStream(f));
     zxid = playLog(BinaryInputArchive.getArchive(logIs));
     logIs.close();
   }
   return zxid;
 }
 /**
  * create the old snapshot database apply logs to it and create the final database
  *
  * @throws IOException
  */
 private void loadThisSnapShot() throws IOException {
   // pick the most recent snapshot
   File snapshot = findMostRecentSnapshot();
   if (snapshot == null) {
     throw new IOException("Invalid snapshots " + "or not snapshots in " + snapShotDir);
   }
   InputStream inputstream = new BufferedInputStream(new FileInputStream(snapshot));
   InputArchive ia = BinaryInputArchive.getArchive(inputstream);
   deserializeSnapshot(oldDataTree, ia, sessionsWithTimeouts);
   // ok done with the snapshot
   // now apply the logs
   long snapshotZxid = oldDataTree.lastProcessedZxid;
   File[] files = FileTxnLog.getLogFiles(dataDir.listFiles(), snapshotZxid);
   long zxid = processLogFiles(oldDataTree, files);
   // check for this zxid to be sane
   if (zxid != oldDataTree.lastProcessedZxid) {
     LOG.error(
         "Zxids not equal "
             + " log zxid "
             + zxid
             + " datatree processed "
             + oldDataTree.lastProcessedZxid);
   }
 }
 /**
  * play the log from this logstream into the datatree
  *
  * @param logStream
  * @return
  * @throws IOException
  */
 public long playLog(InputArchive logStream) throws IOException {
   long highestZxid = 0;
   try {
     while (true) {
       byte[] bytes = logStream.readBuffer("txnEntry");
       if (bytes.length == 0) {
         // Since we preallocate, we define EOF to be an
         // empty transaction
         throw new EOFException();
       }
       InputArchive ia = BinaryInputArchive.getArchive(new ByteArrayInputStream(bytes));
       TxnHeader hdr = new TxnHeader();
       Record txn = SerializeUtils.deserializeTxn(ia, hdr);
       if (logStream.readByte("EOR") != 'B') {
         LOG.warn("Last transaction was partial.");
         throw new EOFException("Last transaction was partial.");
       }
       if (hdr.getZxid() <= highestZxid && highestZxid != 0) {
         LOG.error(
             highestZxid
                 + "(higestZxid) >= "
                 + hdr.getZxid()
                 + "(next log) for type "
                 + hdr.getType());
       } else {
         highestZxid = hdr.getZxid();
       }
       switch (hdr.getType()) {
         case OpCode.createSession:
           sessionsWithTimeouts.put(hdr.getClientId(), ((CreateSessionTxn) txn).getTimeOut());
           if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(
                 LOG,
                 ZooTrace.SESSION_TRACE_MASK,
                 "playLog --- create session in log: 0x"
                     + Long.toHexString(hdr.getClientId())
                     + " with timeout: "
                     + ((CreateSessionTxn) txn).getTimeOut());
           }
           // give dataTree a chance to sync its lastProcessedZxid
           oldDataTree.processTxn(hdr, txn);
           break;
         case OpCode.closeSession:
           sessionsWithTimeouts.remove(hdr.getClientId());
           if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(
                 LOG,
                 ZooTrace.SESSION_TRACE_MASK,
                 "playLog --- close session in log: 0x" + Long.toHexString(hdr.getClientId()));
           }
           oldDataTree.processTxn(hdr, txn);
           break;
         default:
           oldDataTree.processTxn(hdr, txn);
       }
       Request r = new Request(null, 0, hdr.getCxid(), hdr.getType(), null, null);
       r.txn = txn;
       r.hdr = hdr;
       r.zxid = hdr.getZxid();
     }
   } catch (EOFException e) {
     // expected in some cases - see comments in try block
   }
   return highestZxid;
 }
 public static void byteBuffer2Record(ByteBuffer bb, Record record) throws IOException {
   BinaryInputArchive ia;
   ia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
   record.deserialize(ia, "request");
 }
  /**
   * This thread will receive packets from the peer and process them and also listen to new
   * connections from new peers.
   */
  @Override
  public void run() {
    try {

      ia = BinaryInputArchive.getArchive(new BufferedInputStream(sock.getInputStream()));
      bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
      oa = BinaryOutputArchive.getArchive(bufferedOutput);

      QuorumPacket qp = new QuorumPacket();
      ia.readRecord(qp, "packet");
      if (qp.getType() != Leader.FOLLOWERINFO && qp.getType() != Leader.OBSERVERINFO) {
        LOG.error("First packet " + qp.toString() + " is not FOLLOWERINFO or OBSERVERINFO!");
        return;
      }
      if (qp.getData() != null) {
        ByteBuffer bbsid = ByteBuffer.wrap(qp.getData());
        this.sid = bbsid.getLong();
      } else {
        this.sid = leader.followerCounter.getAndDecrement();
      }

      LOG.info("Follower sid: " + this.sid + " : info : " + leader.self.quorumPeers.get(this.sid));

      if (qp.getType() == Leader.OBSERVERINFO) {
        learnerType = LearnerType.OBSERVER;
      }

      long peerLastZxid = qp.getZxid();
      /* the default to send to the follower */
      int packetToSend = Leader.SNAP;
      long zxidToSend = 0;
      long leaderLastZxid = 0;
      /** the packets that the follower needs to get updates from * */
      long updates = peerLastZxid;

      /* we are sending the diff check if we have proposals in memory to be able to
       * send a diff to the
       */
      ReentrantReadWriteLock lock = leader.zk.getZKDatabase().getLogLock();
      ReadLock rl = lock.readLock();
      try {
        rl.lock();
        final long maxCommittedLog = leader.zk.getZKDatabase().getmaxCommittedLog();
        final long minCommittedLog = leader.zk.getZKDatabase().getminCommittedLog();
        LOG.info(
            "Synchronizing with Follower sid: "
                + this.sid
                + " maxCommittedLog ="
                + Long.toHexString(maxCommittedLog)
                + " minCommittedLog = "
                + Long.toHexString(minCommittedLog)
                + " peerLastZxid = "
                + Long.toHexString(peerLastZxid));

        LinkedList<Proposal> proposals = leader.zk.getZKDatabase().getCommittedLog();

        if (proposals.size() != 0) {
          if ((maxCommittedLog >= peerLastZxid) && (minCommittedLog <= peerLastZxid)) {

            // as we look through proposals, this variable keeps track of previous
            // proposal Id.
            long prevProposalZxid = minCommittedLog;

            // Keep track of whether we are about to send the first packet.
            // Before sending the first packet, we have to tell the learner
            // whether to expect a trunc or a diff
            boolean firstPacket = true;

            for (Proposal propose : proposals) {
              // skip the proposals the peer already has
              if (propose.packet.getZxid() <= peerLastZxid) {
                prevProposalZxid = propose.packet.getZxid();
                continue;
              } else {
                // If we are sending the first packet, figure out whether to trunc
                // in case the follower has some proposals that the leader doesn't
                if (firstPacket) {
                  firstPacket = false;
                  // Does the peer have some proposals that the leader hasn't seen yet
                  if (prevProposalZxid < peerLastZxid) {
                    // send a trunc message before sending the diff
                    packetToSend = Leader.TRUNC;
                    LOG.info("Sending TRUNC");
                    zxidToSend = prevProposalZxid;
                    updates = zxidToSend;
                  } else {
                    // Just send the diff
                    packetToSend = Leader.DIFF;
                    LOG.info("Sending diff");
                    zxidToSend = maxCommittedLog;
                  }
                }
                queuePacket(propose.packet);
                QuorumPacket qcommit =
                    new QuorumPacket(Leader.COMMIT, propose.packet.getZxid(), null, null);
                queuePacket(qcommit);
              }
            }
          } else if (peerLastZxid > maxCommittedLog) {
            packetToSend = Leader.TRUNC;
            zxidToSend = maxCommittedLog;
            updates = zxidToSend;
          }
        } else {
          // just let the state transfer happen
        }

        leaderLastZxid = leader.startForwarding(this, updates);
        if (peerLastZxid == leaderLastZxid) {
          // We are in sync so we'll do an empty diff
          packetToSend = Leader.DIFF;
          zxidToSend = leaderLastZxid;
        }
      } finally {
        rl.unlock();
      }

      QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER, leaderLastZxid, null, null);
      oa.writeRecord(newLeaderQP, "packet");
      bufferedOutput.flush();
      // Need to set the zxidToSend to the latest zxid
      if (packetToSend == Leader.SNAP) {
        zxidToSend = leader.zk.getZKDatabase().getDataTreeLastProcessedZxid();
      }
      oa.writeRecord(new QuorumPacket(packetToSend, zxidToSend, null, null), "packet");
      bufferedOutput.flush();

      /* if we are not truncating or sending a diff just send a snapshot */
      if (packetToSend == Leader.SNAP) {
        LOG.info(
            "Sending snapshot last zxid of peer is 0x"
                + Long.toHexString(peerLastZxid)
                + " "
                + " zxid of leader is 0x"
                + Long.toHexString(leaderLastZxid)
                + "sent zxid of db as 0x"
                + Long.toHexString(zxidToSend));
        // Dump data to peer
        leader.zk.getZKDatabase().serializeSnapshot(oa);
        oa.writeString("BenWasHere", "signature");
      }
      bufferedOutput.flush();

      // Mutation packets will be queued during the serialize,
      // so we need to mark when the peer can actually start
      // using the data
      //
      queuedPackets.add(new QuorumPacket(Leader.UPTODATE, -1, null, null));

      // Start sending packets
      new Thread() {
        public void run() {
          Thread.currentThread().setName("Sender-" + sock.getRemoteSocketAddress());
          try {
            sendPackets();
          } catch (InterruptedException e) {
            LOG.warn("Unexpected interruption", e);
          }
        }
      }.start();

      /*
       * Have to wait for the first ACK, wait until
       * the leader is ready, and only then we can
       * start processing messages.
       */
      qp = new QuorumPacket();
      ia.readRecord(qp, "packet");
      if (qp.getType() != Leader.ACK) {
        LOG.error("Next packet was supposed to be an ACK");
        return;
      }
      leader.processAck(this.sid, qp.getZxid(), sock.getLocalSocketAddress());

      /*
       * Wait until leader starts up
       */
      synchronized (leader.zk) {
        while (!leader.zk.isRunning()) {
          leader.zk.wait(500);
        }
      }

      while (true) {
        qp = new QuorumPacket();
        ia.readRecord(qp, "packet");

        long traceMask = ZooTrace.SERVER_PACKET_TRACE_MASK;
        if (qp.getType() == Leader.PING) {
          traceMask = ZooTrace.SERVER_PING_TRACE_MASK;
        }
        if (LOG.isTraceEnabled()) {
          ZooTrace.logQuorumPacket(LOG, traceMask, 'i', qp);
        }
        tickOfLastAck = leader.self.tick;

        ByteBuffer bb;
        long sessionId;
        int cxid;
        int type;

        switch (qp.getType()) {
          case Leader.ACK:
            if (this.learnerType == LearnerType.OBSERVER) {
              if (LOG.isDebugEnabled()) {
                LOG.debug("Received ACK from Observer  " + this.sid);
              }
            }
            leader.processAck(this.sid, qp.getZxid(), sock.getLocalSocketAddress());
            break;
          case Leader.PING:
            // Process the touches
            ByteArrayInputStream bis = new ByteArrayInputStream(qp.getData());
            DataInputStream dis = new DataInputStream(bis);
            while (dis.available() > 0) {
              long sess = dis.readLong();
              int to = dis.readInt();
              leader.zk.touch(sess, to);
            }
            break;
          case Leader.REVALIDATE:
            bis = new ByteArrayInputStream(qp.getData());
            dis = new DataInputStream(bis);
            long id = dis.readLong();
            int to = dis.readInt();
            ByteArrayOutputStream bos = new ByteArrayOutputStream();
            DataOutputStream dos = new DataOutputStream(bos);
            dos.writeLong(id);
            boolean valid = leader.zk.touch(id, to);
            if (valid) {
              try {
                // set the session owner
                // as the follower that
                // owns the session
                leader.zk.setOwner(id, this);
              } catch (SessionExpiredException e) {
                LOG.error(
                    "Somehow session "
                        + Long.toHexString(id)
                        + " expired right after being renewed! (impossible)",
                    e);
              }
            }
            if (LOG.isTraceEnabled()) {
              ZooTrace.logTraceMessage(
                  LOG,
                  ZooTrace.SESSION_TRACE_MASK,
                  "Session 0x" + Long.toHexString(id) + " is valid: " + valid);
            }
            dos.writeBoolean(valid);
            qp.setData(bos.toByteArray());
            queuedPackets.add(qp);
            break;
          case Leader.REQUEST:
            bb = ByteBuffer.wrap(qp.getData());
            sessionId = bb.getLong();
            cxid = bb.getInt();
            type = bb.getInt();
            bb = bb.slice();
            Request si;
            if (type == OpCode.sync) {
              si = new LearnerSyncRequest(this, sessionId, cxid, type, bb, qp.getAuthinfo());
            } else {
              si = new Request(null, sessionId, cxid, type, bb, qp.getAuthinfo());
            }
            si.setOwner(this);
            leader.zk.submitRequest(si);
            break;
          default:
        }
      }
    } catch (IOException e) {
      if (sock != null && !sock.isClosed()) {
        LOG.error("Unexpected exception causing shutdown while sock " + "still open", e);
        // close the socket to make sure the
        // other side can see it being close
        try {
          sock.close();
        } catch (IOException ie) {
          // do nothing
        }
      }
    } catch (InterruptedException e) {
      LOG.error("Unexpected exception causing shutdown", e);
    } finally {
      LOG.warn(
          "******* GOODBYE "
              + (sock != null ? sock.getRemoteSocketAddress() : "<null>")
              + " ********");
      // Send the packet of death
      try {
        queuedPackets.put(proposalOfDeath);
      } catch (InterruptedException e) {
        LOG.warn("Ignoring unexpected exception", e);
      }
      shutdown();
    }
  }
  public static String packetToString(QuorumPacket p) {
    if (true) return null;
    String type = null;
    String mess = null;
    Record txn = null;

    switch (p.getType()) {
      case Leader.ACK:
        type = "ACK";
        break;
      case Leader.COMMIT:
        type = "COMMIT";
        break;
      case Leader.FOLLOWERINFO:
        type = "FOLLOWERINFO";
        break;
      case Leader.NEWLEADER:
        type = "NEWLEADER";
        break;
      case Leader.PING:
        type = "PING";
        break;
      case Leader.PROPOSAL:
        type = "PROPOSAL";
        BinaryInputArchive ia =
            BinaryInputArchive.getArchive(new ByteArrayInputStream(p.getData()));
        TxnHeader hdr = new TxnHeader();
        try {
          txn = SerializeUtils.deserializeTxn(ia, hdr);
          // mess = "transaction: " + txn.toString();
        } catch (IOException e) {
          LOG.warn("Unexpected exception", e);
        }
        break;
      case Leader.REQUEST:
        type = "REQUEST";
        break;
      case Leader.REVALIDATE:
        type = "REVALIDATE";
        ByteArrayInputStream bis = new ByteArrayInputStream(p.getData());
        DataInputStream dis = new DataInputStream(bis);
        try {
          long id = dis.readLong();
          mess = " sessionid = " + id;
        } catch (IOException e) {
          LOG.warn("Unexpected exception", e);
        }

        break;
      case Leader.UPTODATE:
        type = "UPTODATE";
        break;
      default:
        type = "UNKNOWN" + p.getType();
    }
    String entry = null;
    if (type != null) {
      entry = type + " " + Long.toHexString(p.getZxid()) + " " + mess;
    }
    return entry;
  }