@Override
    public <R> OperationResult<R> execute(Operation<Cassandra.Client, R> op)
        throws ConnectionException {
      long startTime = System.nanoTime();
      long latency = 0;
      setTimeout(cpConfig.getSocketTimeout()); // In case the configurationchanged
      operationCounter.incrementAndGet();

      // Set a new keyspace, if it changed
      lastException = null;
      if (op.getKeyspace() != null
          && (keyspaceName == null || !op.getKeyspace().equals(keyspaceName))) {
        CassandraOperationTracer tracer =
            tracerFactory.newTracer(CassandraOperationType.SET_KEYSPACE).start();
        try {
          cassandraClient.set_keyspace(op.getKeyspace());
          if (asConfig.getCqlVersion() != null)
            cassandraClient.set_cql_version(asConfig.getCqlVersion());
          keyspaceName = op.getKeyspace();
          long now = System.nanoTime();
          latency = now - startTime;
          pool.addLatencySample(latency, now);
          tracer.success();
        } catch (Exception e) {
          long now = System.nanoTime();
          latency = now - startTime;
          lastException = ThriftConverter.ToConnectionPoolException(e).setLatency(latency);
          if (e instanceof IsTimeoutException) {
            pool.addLatencySample(
                TimeUnit.NANOSECONDS.convert(cpConfig.getSocketTimeout(), TimeUnit.MILLISECONDS),
                now);
          }
          tracer.failure(lastException);
          throw lastException;
        }
        startTime = System.nanoTime(); // We don't want to include
        // the set_keyspace in our
        // latency calculation
      }

      // Execute the operation
      try {
        R result = op.execute(cassandraClient, this);
        long now = System.nanoTime();
        latency = now - startTime;
        pool.addLatencySample(latency, now);
        return new OperationResultImpl<R>(getHost(), result, latency);
      } catch (Exception e) {
        long now = System.nanoTime();
        latency = now - startTime;
        lastException = ThriftConverter.ToConnectionPoolException(e).setLatency(latency);
        if (e instanceof IsTimeoutException) {
          pool.addLatencySample(
              TimeUnit.NANOSECONDS.convert(cpConfig.getSocketTimeout(), TimeUnit.MILLISECONDS),
              now);
        }
        throw lastException;
      }
    }
 @Override
 public Host getHost() {
   return pool.getHost();
 }
    @Override
    public void open() throws ConnectionException {
      if (cassandraClient != null) {
        throw new IllegalStateException("Open called on already open connection");
      }

      long startTime = System.currentTimeMillis();
      try {
        final SSLConnectionContext sslCxt = cpConfig.getSSLConnectionContext();
        if (sslCxt != null) {
          TSSLTransportParameters params =
              new TSSLTransportParameters(
                  sslCxt.getSslProtocol(), sslCxt.getSslCipherSuites().toArray(new String[0]));
          params.setTrustStore(sslCxt.getSslTruststore(), sslCxt.getSslTruststorePassword());
          // thrift's SSL implementation does not allow you set the socket connect timeout, only
          // read timeout
          socket =
              TSSLTransportFactory.getClientSocket(
                  getHost().getIpAddress(),
                  getHost().getPort(),
                  cpConfig.getSocketTimeout(),
                  params);
        } else {
          socket =
              new TSocket(
                  getHost().getIpAddress(), getHost().getPort(), cpConfig.getConnectTimeout());
        }

        socket.getSocket().setTcpNoDelay(true);
        socket.getSocket().setKeepAlive(true);
        socket.getSocket().setSoLinger(false, 0);

        setTimeout(cpConfig.getSocketTimeout());
        transport = new TFramedTransport(socket);
        if (!transport.isOpen()) transport.open();

        cassandraClient =
            new Cassandra.Client(new TBinaryProtocol.Factory().getProtocol(transport));
        monitor.incConnectionCreated(getHost());

        AuthenticationCredentials credentials = cpConfig.getAuthenticationCredentials();
        if (credentials != null) {
          Map<String, String> thriftCredentials = Maps.newHashMapWithExpectedSize(2);
          thriftCredentials.put("username", credentials.getUsername());
          thriftCredentials.put("password", credentials.getPassword());
          cassandraClient.login(new AuthenticationRequest(thriftCredentials));
        }
      } catch (Exception e) {
        pool.addLatencySample(
            TimeUnit.NANOSECONDS.convert(cpConfig.getSocketTimeout(), TimeUnit.MILLISECONDS),
            System.nanoTime());
        closeClient();
        ConnectionException ce =
            ThriftConverter.ToConnectionPoolException(e)
                .setHost(getHost())
                .setLatency(System.currentTimeMillis() - startTime);
        monitor.incConnectionCreateFailed(getHost(), ce);
        throw ce;
      } catch (Throwable t) {
        LOG.error("Error creating connection", t);
        pool.addLatencySample(
            TimeUnit.NANOSECONDS.convert(cpConfig.getSocketTimeout(), TimeUnit.MILLISECONDS),
            System.nanoTime());
        closeClient();
        ConnectionException ce =
            ThriftConverter.ToConnectionPoolException(
                    new RuntimeException("Error openning connection", t))
                .setHost(getHost())
                .setLatency(System.currentTimeMillis() - startTime);
        monitor.incConnectionCreateFailed(getHost(), ce);
        throw ce;
      }
    }
  @SuppressWarnings("unchecked")
  @Override
  /**
   * Update the list of pools using the provided mapping of start token to collection of hosts that
   * own the token
   */
  public synchronized boolean setPools(Collection<HostConnectionPool<CL>> ring) {
    boolean didChange = false;

    Set<HostConnectionPool<CL>> allPools = Sets.newHashSet();

    // Create a mapping of end token to a list of hosts that own the token
    Map<RingPosition, List<HostConnectionPool<CL>>> tokenHostMap = Maps.newHashMap();
    for (HostConnectionPool<CL> pool : ring) {
      allPools.add(pool);
      if (!this.allPools.hasPool(pool)) didChange = true;

      for (TokenRange range : pool.getHost().getTokenRanges()) {
        RingPosition endToken = partitioner.getRingPositionForToken(range.getEndToken());
        List<HostConnectionPool<CL>> partition = tokenHostMap.get(endToken);
        if (partition == null) {
          partition = Lists.newArrayList();
          tokenHostMap.put(endToken, partition);
        }
        partition.add(pool);
      }
    }

    // Temporary list of token that will be removed if not found in the new ring
    Set<RingPosition> tokensToRemove = Sets.newHashSet(tokenHostMap.keySet());

    // Iterate all tokens
    for (Entry<RingPosition, List<HostConnectionPool<CL>>> entry : tokenHostMap.entrySet()) {
      RingPosition token = entry.getKey();
      tokensToRemove.remove(token);

      // Add a new partition or modify an existing one
      TokenHostConnectionPoolPartition<CL> partition = tokenToPartitionMap.get(token);
      if (partition == null) {
        partition = makePartition(token);
        tokenToPartitionMap.put(token, partition);
        didChange = true;
      }
      if (partition.setPools(entry.getValue())) didChange = true;
    }

    // Remove the tokens that are no longer in the ring
    for (RingPosition token : tokensToRemove) {
      tokenHostMap.remove(token);
      didChange = true;
    }

    // Sort partitions by token
    if (didChange) {
      List<TokenHostConnectionPoolPartition<CL>> partitions =
          Lists.newArrayList(tokenToPartitionMap.values());
      Collections.sort(partitions, partitionComparator);
      this.allPools.setPools(allPools);
      refresh();
      this.sortedRing.set(Collections.unmodifiableList(partitions));
    }

    return didChange;
  }