/**
   * If the key is to be associated with a valid value, a mutation is created for it with the given
   * table and columns. In the event the value in the column is missing (i.e., null), then it is
   * marked for {@link Deletion}. Similarly, if the entire value for a key is missing (i.e., null),
   * then the entire key is marked for {@link Deletion}.
   *
   * @param keyColumns the key to write.
   * @param values the values to write.
   * @throws IOException
   */
  @Override
  public void write(Map<String, ByteBuffer> keyColumns, List<ByteBuffer> values)
      throws IOException {
    TokenRange range = ringCache.getRange(getPartitionKey(keyColumns));

    // get the client for the given range, or create a new one
    final InetAddress address = ringCache.getEndpoints(range).get(0);
    RangeClient client = clients.get(address);
    if (client == null) {
      // haven't seen keys for this range: create new client
      client = new RangeClient(ringCache.getEndpoints(range));
      client.start();
      clients.put(address, client);
    }

    // add primary key columns to the bind variables
    List<ByteBuffer> allValues = new ArrayList<ByteBuffer>(values);
    for (ColumnMetadata column : partitionKeyColumns)
      allValues.add(keyColumns.get(column.getName()));
    for (ColumnMetadata column : clusterColumns) allValues.add(keyColumns.get(column.getName()));

    client.put(allValues);

    if (progressable != null) progressable.progress();
    if (context != null) HadoopCompat.progress(context);
  }
 public List<InetAddress> getEndpoints(TokenRange range) {
   Set<Host> hostSet = rangeMap.get(range);
   List<InetAddress> addresses = new ArrayList<>(hostSet.size());
   for (Host host : hostSet) {
     addresses.add(host.getAddress());
   }
   return addresses;
 }
Esempio n. 3
0
  @Test
  public void executePreparedCounterTest() throws Exception {
    PreparedStatement p = session.prepare("UPDATE " + COUNTER_TABLE + " SET c = c + ? WHERE k = ?");

    session.execute(p.bind(1L, "row"));
    session.execute(p.bind(1L, "row"));

    ResultSet rs = session.execute("SELECT * FROM " + COUNTER_TABLE);
    List<Row> rows = rs.all();
    assertEquals(1, rows.size());
    assertEquals(2L, rows.get(0).getLong("c"));
  }
  private ByteBuffer getPartitionKey(Map<String, ByteBuffer> keyColumns) {
    ByteBuffer partitionKey;
    if (partitionKeyColumns.size() > 1) {
      ByteBuffer[] keys = new ByteBuffer[partitionKeyColumns.size()];
      for (int i = 0; i < keys.length; i++)
        keys[i] = keyColumns.get(partitionKeyColumns.get(i).getName());

      partitionKey = CompositeType.build(keys);
    } else {
      partitionKey = keyColumns.get(partitionKeyColumns.get(0).getName());
    }
    return partitionKey;
  }
    /** Loops collecting cql binded variable values from the queue and sending to Cassandra */
    public void run() {
      outer:
      while (run || !queue.isEmpty()) {
        List<ByteBuffer> bindVariables;
        try {
          bindVariables = queue.take();
        } catch (InterruptedException e) {
          // re-check loop condition after interrupt
          continue;
        }

        ListIterator<InetAddress> iter = endpoints.listIterator();
        while (true) {
          // send the mutation to the last-used endpoint.  first time through, this will NPE
          // harmlessly.
          try {
            int i = 0;
            PreparedStatement statement = preparedStatement(client);
            while (bindVariables != null) {
              BoundStatement boundStatement = new BoundStatement(statement);
              for (int columnPosition = 0;
                  columnPosition < bindVariables.size();
                  columnPosition++) {
                boundStatement.setBytesUnsafe(columnPosition, bindVariables.get(columnPosition));
              }
              client.execute(boundStatement);
              i++;

              if (i >= batchThreshold) break;
              bindVariables = queue.poll();
            }
            break;
          } catch (Exception e) {
            closeInternal();
            if (!iter.hasNext()) {
              lastException = new IOException(e);
              break outer;
            }
          }

          // attempt to connect to a different endpoint
          try {
            InetAddress address = iter.next();
            String host = address.getHostName();
            client = CqlConfigHelper.getOutputCluster(host, conf).connect();
          } catch (Exception e) {
            // If connection died due to Interrupt, just try connecting to the endpoint again.
            // There are too many ways for the Thread.interrupted() state to be cleared, so
            // we can't rely on that here. Until the java driver gives us a better way of knowing
            // that this exception came from an InterruptedException, this is the best solution.
            if (canRetryDriverConnection(e)) {
              iter.previous();
            }
            closeInternal();

            // Most exceptions mean something unexpected went wrong to that endpoint, so
            // we should try again to another.  Other exceptions (auth or invalid request) are
            // fatal.
            if ((e instanceof AuthenticationException || e instanceof InvalidQueryException)
                || !iter.hasNext()) {
              lastException = new IOException(e);
              break outer;
            }
          }
        }
      }
      // close all our connections once we are done.
      closeInternal();
    }