@Override
  public Map<Integer, RootOperator[]> getWorkerPlan(int[] allWorkers) throws Exception {

    final DbQueryScan localScan =
        new DbQueryScan("select sourceIPAddr, adRevenue from UserVisits", scanSchema);

    final SingleGroupByAggregate localAgg =
        new SingleGroupByAggregate(
            localScan, 0, new SingleColumnAggregatorFactory(1, AggregationOp.SUM));

    final ExchangePairID shuffleLocalGroupByID = ExchangePairID.newID();

    PartitionFunction pf = new SingleFieldHashPartitionFunction(allWorkers.length, 0);

    final GenericShuffleProducer shuffleLocalGroupBy =
        new GenericShuffleProducer(localAgg, shuffleLocalGroupByID, allWorkers, pf);
    final GenericShuffleConsumer sc =
        new GenericShuffleConsumer(
            shuffleLocalGroupBy.getSchema(), shuffleLocalGroupByID, allWorkers);

    final SingleGroupByAggregate globalAgg =
        new SingleGroupByAggregate(sc, 0, new SingleColumnAggregatorFactory(1, AggregationOp.SUM));

    final CollectProducer sendToMaster = new CollectProducer(globalAgg, sendToMasterID, 0);

    final Map<Integer, RootOperator[]> result = new HashMap<Integer, RootOperator[]>();
    for (int worker : allWorkers) {
      result.put(worker, new RootOperator[] {shuffleLocalGroupBy, sendToMaster});
    }

    return result;
  }
public class AggregateQuerySQLiteMyriaAggregate implements QueryPlanGenerator {

  /** */
  private static final long serialVersionUID = 5147615855012257218L;

  static final ImmutableList<Type> outputTypes =
      ImmutableList.of(Type.STRING_TYPE, Type.DOUBLE_TYPE);
  static final ImmutableList<String> outputColumnNames =
      ImmutableList.of("sourceIPAddr", "sum_adRevenue");
  static final Schema outputSchema = new Schema(outputTypes, outputColumnNames);

  static final ImmutableList<Type> scanTypes = ImmutableList.of(Type.STRING_TYPE, Type.DOUBLE_TYPE);
  static final ImmutableList<String> scanColumnNames =
      ImmutableList.of("sourceIPAddr", "adRevenue");
  static final Schema scanSchema = new Schema(scanTypes, scanColumnNames);

  final ExchangePairID sendToMasterID = ExchangePairID.newID();

  @Override
  public Map<Integer, RootOperator[]> getWorkerPlan(int[] allWorkers) throws Exception {

    final DbQueryScan localScan =
        new DbQueryScan("select sourceIPAddr, adRevenue from UserVisits", scanSchema);

    final SingleGroupByAggregate localAgg =
        new SingleGroupByAggregate(
            localScan, 0, new SingleColumnAggregatorFactory(1, AggregationOp.SUM));

    final ExchangePairID shuffleLocalGroupByID = ExchangePairID.newID();

    PartitionFunction pf = new SingleFieldHashPartitionFunction(allWorkers.length, 0);

    final GenericShuffleProducer shuffleLocalGroupBy =
        new GenericShuffleProducer(localAgg, shuffleLocalGroupByID, allWorkers, pf);
    final GenericShuffleConsumer sc =
        new GenericShuffleConsumer(
            shuffleLocalGroupBy.getSchema(), shuffleLocalGroupByID, allWorkers);

    final SingleGroupByAggregate globalAgg =
        new SingleGroupByAggregate(sc, 0, new SingleColumnAggregatorFactory(1, AggregationOp.SUM));

    final CollectProducer sendToMaster = new CollectProducer(globalAgg, sendToMasterID, 0);

    final Map<Integer, RootOperator[]> result = new HashMap<Integer, RootOperator[]>();
    for (int worker : allWorkers) {
      result.put(worker, new RootOperator[] {shuffleLocalGroupBy, sendToMaster});
    }

    return result;
  }

  @Override
  public SinkRoot getMasterPlan(
      int[] allWorkers, final LinkedBlockingQueue<TupleBatch> receivedTupleBatches) {
    final CollectConsumer serverCollect =
        new CollectConsumer(outputSchema, sendToMasterID, allWorkers);
    SinkRoot serverPlan = new SinkRoot(serverCollect);
    return serverPlan;
  }
}
Exemple #3
0
  @Override
  public Map<Integer, RootOperator[]> getWorkerPlan(int[] allWorkers) throws Exception {
    final ExchangePairID collectCountID = ExchangePairID.newID();

    final DbQueryScan countDictionary =
        new DbQueryScan("select count(*),0 from Dictionary", countSchema);
    final DbQueryScan countTriples = new DbQueryScan("select count(*),0 from Triples", countSchema);

    final SymmetricHashJoin countMergeJoin =
        new SymmetricHashJoin(countDictionary, countTriples, new int[] {1}, new int[] {1});

    final CollectProducer collectCountP =
        new CollectProducer(countMergeJoin, collectCountID, allWorkers[0]);
    final CollectConsumer collectCountC =
        new CollectConsumer(collectCountP.getSchema(), collectCountID, allWorkers);

    final Aggregate agg =
        new Aggregate(
            collectCountC,
            new AggregatorFactory[] {
              new SingleColumnAggregatorFactory(0, AggregationOp.SUM),
              new SingleColumnAggregatorFactory(2, AggregationOp.SUM)
            });

    final CollectProducer sendToMaster = new CollectProducer(agg, sendToMasterID, 0);

    final Map<Integer, RootOperator[]> result = new HashMap<Integer, RootOperator[]>();
    result.put(allWorkers[0], new RootOperator[] {sendToMaster, collectCountP});

    for (int i = 1; i < allWorkers.length; i++) {
      result.put(allWorkers[i], new RootOperator[] {collectCountP});
    }

    return result;
  }
Exemple #4
0
  /**
   * Actually allocate the real operator IDs and real worker IDs for the producers and consumers.
   *
   * @see #assignWorkersToFragments(List, ConstructArgs)
   * @param fragments the fragments of the plan
   */
  private static void fillInRealOperatorAndWorkerIDs(final List<PlanFragmentEncoding> fragments) {
    Multimap<Integer, ExchangePairID> consumerMap = ArrayListMultimap.create();
    Map<Integer, List<Integer>> producerWorkerMap = Maps.newHashMap();
    Map<Integer, List<Integer>> consumerWorkerMap = Maps.newHashMap();

    /*
     * First pass: create a new ExchangePairID for each Consumer, and set it. Also track the workers for each producer
     * and consumer.
     */
    for (PlanFragmentEncoding fragment : fragments) {
      for (OperatorEncoding<?> operator : fragment.operators) {
        if (operator instanceof AbstractConsumerEncoding<?>) {
          AbstractConsumerEncoding<?> consumer = (AbstractConsumerEncoding<?>) operator;
          ExchangePairID exchangeId = ExchangePairID.newID();
          consumerMap.put(consumer.argOperatorId, exchangeId);
          consumerWorkerMap.put(consumer.argOperatorId, fragment.workers);
          consumer.setRealOperatorIds(ImmutableList.of(exchangeId));
        } else if (operator instanceof AbstractProducerEncoding<?>
            || operator instanceof IDBControllerEncoding) {
          producerWorkerMap.put(operator.opId, fragment.workers);
        }
      }
    }

    /* Second pass: set the ExchangePairIDs for each producer, also the workers for these and the consumers. */
    for (PlanFragmentEncoding fragment : fragments) {
      for (OperatorEncoding<?> operator : fragment.operators) {
        if (operator instanceof AbstractConsumerEncoding<?>) {
          AbstractConsumerEncoding<?> consumer = (AbstractConsumerEncoding<?>) operator;
          consumer.setRealWorkerIds(
              ImmutableSet.copyOf(producerWorkerMap.get(consumer.argOperatorId)));
        } else if (operator instanceof AbstractProducerEncoding<?>) {
          AbstractProducerEncoding<?> producer = (AbstractProducerEncoding<?>) operator;
          producer.setRealWorkerIds(ImmutableSet.copyOf(consumerWorkerMap.get(producer.opId)));
          producer.setRealOperatorIds(ImmutableList.copyOf(consumerMap.get(producer.opId)));
        } else if (operator instanceof IDBControllerEncoding) {
          IDBControllerEncoding idbController = (IDBControllerEncoding) operator;
          idbController.realEosControllerWorkerId =
              consumerWorkerMap.get(idbController.opId).get(0);
          idbController.setRealEosControllerOperatorID(
              consumerMap.get(idbController.opId).iterator().next());
        }
      }
    }
  }
Exemple #5
0
  /**
   * Builds the query plan to update the {@link Server}'s master catalog with the number of tuples
   * in every relation written by a subquery. The query plan is basically "SELECT RelationKey,
   * COUNT(*)" -> Collect at master -> "SELECT RelationKey, SUM(counts)".
   *
   * @param relationsWritten the metadata about which relations were written during the execution of
   *     this subquery.
   * @param server the server on which the catalog will be updated
   * @return the query plan to update the master's catalog with the new number of tuples for all
   *     written relations.
   */
  public static SubQuery getRelationTupleUpdateSubQuery(
      final Map<RelationKey, RelationWriteMetadata> relationsWritten, final Server server) {
    ExchangePairID collectId = ExchangePairID.newID();
    Schema schema =
        Schema.ofFields(
            "userName",
            Type.STRING_TYPE,
            "programName",
            Type.STRING_TYPE,
            "relationName",
            Type.STRING_TYPE,
            "tupleCount",
            Type.LONG_TYPE);

    String dbms = server.getDBMS();
    Preconditions.checkState(
        dbms != null, "Server must have a configured DBMS environment variable");

    /*
     * Worker plans: for each relation, create a {@link DbQueryScan} to get the count, an {@link Apply} to add the
     * {@link RelationKey}, then a {@link CollectProducer} to send the count to the master.
     */
    Map<Integer, SubQueryPlan> workerPlans = Maps.newHashMap();
    for (RelationWriteMetadata meta : relationsWritten.values()) {
      Set<Integer> workers = meta.getWorkers();
      RelationKey relation = meta.getRelationKey();
      for (Integer worker : workers) {
        DbQueryScan localCount =
            new DbQueryScan(
                "SELECT COUNT(*) FROM " + relation.toString(dbms),
                Schema.ofFields("tupleCount", Type.LONG_TYPE));
        List<Expression> expressions =
            ImmutableList.of(
                new Expression(
                    schema.getColumnName(0), new ConstantExpression(relation.getUserName())),
                new Expression(
                    schema.getColumnName(1), new ConstantExpression(relation.getProgramName())),
                new Expression(
                    schema.getColumnName(2), new ConstantExpression(relation.getRelationName())),
                new Expression(schema.getColumnName(3), new VariableExpression(0)));
        Apply addRelationName = new Apply(localCount, expressions);
        CollectProducer producer =
            new CollectProducer(addRelationName, collectId, MyriaConstants.MASTER_ID);
        if (!workerPlans.containsKey(worker)) {
          workerPlans.put(worker, new SubQueryPlan(producer));
        } else {
          workerPlans.get(worker).addRootOp(producer);
        }
      }
    }

    /* Master plan: collect, sum, insert the updates. */
    CollectConsumer consumer = new CollectConsumer(schema, collectId, workerPlans.keySet());
    MultiGroupByAggregate aggCounts =
        new MultiGroupByAggregate(
            consumer, new int[] {0, 1, 2}, new SingleColumnAggregatorFactory(3, AggregationOp.SUM));
    UpdateCatalog catalog = new UpdateCatalog(aggCounts, server);
    SubQueryPlan masterPlan = new SubQueryPlan(catalog);

    return new SubQuery(masterPlan, workerPlans);
  }
Exemple #6
0
public class CountStar implements QueryPlanGenerator {

  static final ImmutableList<Type> outputTypes = ImmutableList.of(Type.LONG_TYPE, Type.LONG_TYPE);
  static final ImmutableList<String> outputColumnNames =
      ImmutableList.of("count(*) Dictionary", "count(*) Triples");
  static final Schema outputSchema = new Schema(outputTypes, outputColumnNames);

  static final ImmutableList<Type> countTypes = ImmutableList.of(Type.LONG_TYPE, Type.LONG_TYPE);
  static final ImmutableList<String> countColumnNames = ImmutableList.of("count", "dummy");

  static final Schema countSchema = new Schema(countTypes, countColumnNames);
  final ExchangePairID sendToMasterID = ExchangePairID.newID();

  @Override
  public Map<Integer, RootOperator[]> getWorkerPlan(int[] allWorkers) throws Exception {
    final ExchangePairID collectCountID = ExchangePairID.newID();

    final DbQueryScan countDictionary =
        new DbQueryScan("select count(*),0 from Dictionary", countSchema);
    final DbQueryScan countTriples = new DbQueryScan("select count(*),0 from Triples", countSchema);

    final SymmetricHashJoin countMergeJoin =
        new SymmetricHashJoin(countDictionary, countTriples, new int[] {1}, new int[] {1});

    final CollectProducer collectCountP =
        new CollectProducer(countMergeJoin, collectCountID, allWorkers[0]);
    final CollectConsumer collectCountC =
        new CollectConsumer(collectCountP.getSchema(), collectCountID, allWorkers);

    final Aggregate agg =
        new Aggregate(
            collectCountC,
            new AggregatorFactory[] {
              new SingleColumnAggregatorFactory(0, AggregationOp.SUM),
              new SingleColumnAggregatorFactory(2, AggregationOp.SUM)
            });

    final CollectProducer sendToMaster = new CollectProducer(agg, sendToMasterID, 0);

    final Map<Integer, RootOperator[]> result = new HashMap<Integer, RootOperator[]>();
    result.put(allWorkers[0], new RootOperator[] {sendToMaster, collectCountP});

    for (int i = 1; i < allWorkers.length; i++) {
      result.put(allWorkers[i], new RootOperator[] {collectCountP});
    }

    return result;
  }

  @Override
  public RootOperator getMasterPlan(
      int[] allWorkers, final LinkedBlockingQueue<TupleBatch> receivedTupleBatches) {
    final CollectConsumer serverCollect =
        new CollectConsumer(outputSchema, sendToMasterID, new int[] {allWorkers[0]});
    TBQueueExporter queueStore = new TBQueueExporter(receivedTupleBatches, serverCollect);
    SinkRoot serverPlan = new SinkRoot(queueStore);
    return serverPlan;
  }

  public static void main(String[] args) throws Exception {
    System.out.println(new CountStar().getWorkerPlan(new int[] {1, 2, 3, 4, 5}));
  }
}