/**
  * Returns a {@code PTable} that contains the unique elements of this collection mapped to a count
  * of their occurrences.
  */
 public static <S> PTable<S, Long> count(PCollection<S> collect) {
   PTypeFamily tf = collect.getTypeFamily();
   return collect
       .parallelDo(
           "Aggregate.count",
           new MapFn<S, Pair<S, Long>>() {
             public Pair<S, Long> map(S input) {
               return Pair.of(input, 1L);
             }
           },
           tf.tableOf(collect.getPType(), tf.longs()))
       .groupByKey()
       .combineValues(Aggregators.SUM_LONGS());
 }
 /**
  * Returns the number of elements in the provided PCollection.
  *
  * @param collect The PCollection whose elements should be counted.
  * @param <S> The type of the PCollection.
  * @return A {@code PObject} containing the number of elements in the {@code PCollection}.
  */
 public static <S> PObject<Long> length(PCollection<S> collect) {
   PTypeFamily tf = collect.getTypeFamily();
   PTable<Integer, Long> countTable =
       collect
           .parallelDo(
               "Aggregate.count",
               new MapFn<S, Pair<Integer, Long>>() {
                 public Pair<Integer, Long> map(S input) {
                   return Pair.of(1, 1L);
                 }
               },
               tf.tableOf(tf.ints(), tf.longs()))
           .groupByKey()
           .combineValues(Aggregators.SUM_LONGS());
   PCollection<Long> count = countTable.values();
   return new FirstElementPObject<Long>(count);
 }
  /** Returns the smallest numerical element from the input collection. */
  public static <S> PObject<S> min(PCollection<S> collect) {
    Class<S> clazz = collect.getPType().getTypeClass();
    if (!clazz.isPrimitive() && !Comparable.class.isAssignableFrom(clazz)) {
      throw new IllegalArgumentException(
          "Can only get min for Comparable elements, not for: "
              + collect.getPType().getTypeClass());
    }
    PTypeFamily tf = collect.getTypeFamily();
    PCollection<S> minCollect =
        PTables.values(
            collect
                .parallelDo(
                    "min",
                    new DoFn<S, Pair<Boolean, S>>() {
                      private transient S min = null;

                      public void process(S input, Emitter<Pair<Boolean, S>> emitter) {
                        if (min == null || ((Comparable<S>) min).compareTo(input) > 0) {
                          min = input;
                        }
                      }

                      public void cleanup(Emitter<Pair<Boolean, S>> emitter) {
                        if (min != null) {
                          emitter.emit(Pair.of(false, min));
                        }
                      }
                    },
                    tf.tableOf(tf.booleans(), collect.getPType()))
                .groupByKey()
                .combineValues(
                    new CombineFn<Boolean, S>() {
                      public void process(
                          Pair<Boolean, Iterable<S>> input, Emitter<Pair<Boolean, S>> emitter) {
                        S min = null;
                        for (S v : input.second()) {
                          if (min == null || ((Comparable<S>) min).compareTo(v) > 0) {
                            min = v;
                          }
                        }
                        emitter.emit(Pair.of(input.first(), min));
                      }
                    }));
    return new FirstElementPObject<S>(minCollect);
  }