@Override
  public void remoteHandler(
      RpcController controller,
      TransactionMapRequest request,
      RpcCallback<TransactionMapResponse> callback) {
    assert (request.hasTransactionId())
        : "Got " + request.getClass().getSimpleName() + " without a txn id!";
    Long txn_id = Long.valueOf(request.getTransactionId());
    if (debug.val)
      LOG.debug(String.format("Got %s for txn #%d", request.getClass().getSimpleName(), txn_id));

    // The mr_ts handle will be null if this HStoreSite is not where the
    // base partition for the original MRTransaction
    MapReduceTransaction mr_ts = hstore_site.getTransaction(txn_id);
    if (mr_ts == null) {
      mr_ts =
          hstore_site
              .getTransactionInitializer()
              .createMapReduceTransaction(
                  txn_id,
                  EstTime.currentTimeMillis(),
                  request.getClientHandle(),
                  request.getBasePartition(),
                  request.getProcedureId(),
                  request.getParams().asReadOnlyByteBuffer());
    }
    assert (mr_ts.isMapPhase());
    mr_ts.initTransactionMapWrapperCallback(callback);

    /*
     * Here we would like to start MapReduce Transaction on the remote partition except the base partition of it.
     * This is to avoid the double invoke for remote task.
     * */
    for (int partition : hstore_site.getLocalPartitionIds()) {
      if (partition != mr_ts.getBasePartition()) {
        LocalTransaction ts = mr_ts.getLocalTransaction(partition);
        hstore_site.transactionStart(ts);
      }
    } // FOR
  }
Пример #2
0
  /** @return */
  public final VoltTable run(Object params[]) {
    assert (this.hstore_site != null)
        : "error in VoltMapReduceProcedure...for hstore_site..........";

    VoltTable result = null;

    // The MapReduceTransaction handle will have all the key information we need about this txn
    long txn_id = this.getTransactionId();
    this.mr_ts = this.hstore_site.getTransaction(txn_id);
    assert (mr_ts != null)
        : "Unexpected null MapReduceTransaction handle for " + this.m_localTxnState;

    // If this invocation is at the txn's base partition, then it is
    // responsible for sending out the coordination messages to the other partitions
    boolean is_local = (this.partitionId == mr_ts.getBasePartition());

    if (mr_ts.isMapPhase()) {
      // If this is the base partition, then we'll send the out the MAP
      // initialization requests to all of the partitions
      if (is_local) {
        // Send out network messages to all other partitions to tell them to
        // execute the MAP phase of this job
        if (debug.get())
          LOG.debug("<VoltMapReduceProcedure.run> is executing ..<Map>...local!!!....\n");
        this.executor
            .getHStoreCoordinator()
            .transactionMap(mr_ts, mr_ts.getTransactionMapCallback());
      }

      this.map_output = mr_ts.getMapOutputByPartition(this.partitionId);
      assert (this.map_output != null);

      if (debug.get()) LOG.debug("<VoltMapReduceProcedure.run> is executing ..<MAP>..\n");
      // Execute the map
      voltQueueSQL(mapInputQuery, params);
      VoltTable mapResult[] = voltExecuteSQLForceSinglePartition();
      assert (mapResult.length == 1);

      // Check whether the HStoreConf flag for locking the entire cluster
      // is true. If it is, then we have to tell the queue manager that we're done.
      // MapReduceTransaction should finish forever...
      if (this.hstore_conf.site.mr_map_blocking) {
        hstore_site.getTransactionQueueManager().lockFinished(txn_id, Status.OK, this.partitionId);
      }

      if (debug.get())
        LOG.debug(
            String.format(
                "MAP: About to process %d records for %s on partition %d",
                mapResult[0].getRowCount(), this.m_localTxnState, this.partitionId));

      if (debug.get())
        LOG.debug(
            String.format("<MapInputTable> Partition:%d\n %s", this.partitionId, mapResult[0]));

      while (mapResult[0].advanceRow()) {
        this.map(mapResult[0].getRow());
      } // WHILE

      if (debug.get())
        LOG.debug(
            String.format(
                "MAP: %s generated %d results on partition %d",
                this.m_localTxnState, this.map_output.getRowCount(), this.partitionId));
      if (debug.get())
        LOG.debug(
            String.format("<MapOutputTable> Partition:%d\n %s", this.partitionId, this.map_output));

      result = mr_ts.getMapOutputByPartition(this.partitionId);

      // Always invoke the TransactionMapWrapperCallback to let somebody know that
      // we finished the MAP phase at this partition
      TransactionMapWrapperCallback callback = mr_ts.getTransactionMapWrapperCallback();
      assert (callback != null) : "Unexpected null callback for " + mr_ts;
      assert (callback.isInitialized()) : "Unexpected uninitalized callback for " + mr_ts;
      callback.run(this.partitionId);
    } else if (mr_ts.isReducePhase()) {
      // If this is the local/base partition, send out the start REDUCE message
      if (is_local) {
        if (debug.get())
          LOG.debug("<VoltMapReduceProcedure.run> is executing ..<Reduce>...local!!!....\n");
        // Send out network messages to all other partitions to tell them to execute the Reduce
        // phase of this job
        this.executor
            .getHStoreCoordinator()
            .transactionReduce(mr_ts, mr_ts.getTransactionReduceCallback());
      }
      this.reduce_input = null; //
      this.reduce_input = mr_ts.getReduceInputByPartition(this.partitionId);
      assert (this.reduce_input != null);
      if (debug.get())
        LOG.debug(
            "__FILE__:__LINE__ "
                + String.format(
                    "TXN: %s, [Stage] \n<VoltMapReduceProcedure.run> is executing <Reduce>..",
                    mr_ts));
      if (debug.get())
        LOG.debug(
            String.format(
                "<ReduceInputTable> Partition:%d\n %s", this.partitionId, this.reduce_input));

      // Sort the the MAP_OUTPUT table
      // Build an "smart" iterator that loops through the MAP_OUTPUT table key-by-key
      VoltTable sorted = VoltTableUtil.sort(this.reduce_input, Pair.of(0, SortDirectionType.ASC));
      // VoltTable sorted = VoltTableUtil.sort(mr_ts.getReduceInputByPartition(this.partitionId),
      // Pair.of(0, SortDirectionType.ASC));
      assert (sorted != null);
      if (debug.get())
        LOG.debug(
            String.format("<Sorted_ReduceInputTable> Partition:%d\n %s", this.partitionId, sorted));

      this.reduce_output = mr_ts.getReduceOutputByPartition(this.partitionId);
      assert (this.reduce_output != null);

      // Make a Hstore.PartitionResult
      ReduceInputIterator<K> rows = new ReduceInputIterator<K>(sorted);

      // Loop over that iterator and call runReduce
      if (debug.get())
        LOG.debug(
            String.format(
                "REDUCE: About to process %d records for %s on partition %d",
                sorted.getRowCount(), this.m_localTxnState, this.partitionId));

      while (rows.hasNext()) {
        K key = rows.getKey();
        this.reduce(key, rows);
      }

      if (debug.get())
        LOG.debug(
            String.format(
                "<ReduceOutputTable> Partition:%d\n %s", this.partitionId, this.reduce_output));

      // Loop over that iterator and call runReduce
      if (debug.get())
        LOG.debug(
            String.format(
                "REDUCE: %s generated %d results on partition %d",
                this.m_localTxnState, this.reduce_output.getRowCount(), this.partitionId));
      ByteString reduceOutData = null;
      try {
        ByteBuffer b = ByteBuffer.wrap(FastSerializer.serialize(reduce_output));
        reduceOutData = ByteString.copyFrom(b.array());
      } catch (Exception ex) {
        throw new RuntimeException(
            String.format(
                "Unexpected error when serializing %s reduceOutput data for partition %d",
                mr_ts, this.partitionId),
            ex);
      }
      ReduceResult.Builder builder =
          ReduceResult.newBuilder()
              .setData(reduceOutData)
              .setPartitionId(this.partitionId)
              .setStatus(Status.OK);

      TransactionReduceWrapperCallback callback = mr_ts.getTransactionReduceWrapperCallback();
      assert (callback != null) : "Unexpected null TransactionReduceWrapperCallback for " + mr_ts;
      assert (callback.isInitialized())
          : "Unexpected uninitalized TransactionReduceWrapperCallback for " + mr_ts;
      callback.run(builder.build());
    }

    return (result);
  }