@Override public PCollection<KV<K, CoGbkResult>> apply(KeyedPCollectionTuple<K> input) { if (input.isEmpty()) { throw new IllegalArgumentException("must have at least one input to a KeyedPCollections"); } // First build the union coder. // TODO: Look at better integration of union types with the // schema specified in the input. List<Coder<?>> codersList = new ArrayList<>(); for (TaggedKeyedPCollection<K, ?> entry : input.getKeyedCollections()) { codersList.add(getValueCoder(entry.pCollection)); } UnionCoder unionCoder = UnionCoder.of(codersList); Coder<K> keyCoder = input.getKeyCoder(); KvCoder<K, RawUnionValue> kVCoder = KvCoder.of(keyCoder, unionCoder); PCollectionList<KV<K, RawUnionValue>> unionTables = PCollectionList.empty(input.getPipeline()); // TODO: Use the schema to order the indices rather than depending // on the fact that the schema ordering is identical to the ordering from // input.getJoinCollections(). int index = -1; for (TaggedKeyedPCollection<K, ?> entry : input.getKeyedCollections()) { index++; PCollection<KV<K, RawUnionValue>> unionTable = makeUnionTable(index, entry.pCollection, kVCoder); unionTables = unionTables.and(unionTable); } PCollection<KV<K, RawUnionValue>> flattenedTable = unionTables.apply(Flatten.<KV<K, RawUnionValue>>pCollections()); PCollection<KV<K, Iterable<RawUnionValue>>> groupedTable = flattenedTable.apply(GroupByKey.<K, RawUnionValue>create()); CoGbkResultSchema tupleTags = input.getCoGbkResultSchema(); PCollection<KV<K, CoGbkResult>> result = groupedTable.apply( ParDo.of(new ConstructCoGbkResultFn<K>(tupleTags)).named("ConstructCoGbkResultFn")); result.setCoder(KvCoder.of(keyCoder, CoGbkResultCoder.of(tupleTags, unionCoder))); return result; }
@Override public PCollection<KV<String, KV<URI, Double>>> apply( PCollection<KV<URI, String>> uriToContent) { // Compute the total number of documents, and // prepare this singleton PCollectionView for // use as a side input. final PCollectionView<Long> totalDocuments = uriToContent .apply(Keys.<URI>create().setName("GetURIs")) .apply(RemoveDuplicates.<URI>create().setName("RemoveDuplicateDocs")) .apply(Count.<URI>globally()) .apply(View.<Long>asSingleton()); // Create a collection of pairs mapping a URI to each // of the words in the document associated with that that URI. PCollection<KV<URI, String>> uriToWords = uriToContent.apply( ParDo.named("SplitWords") .of( new DoFn<KV<URI, String>, KV<URI, String>>() { private static final long serialVersionUID = 0; @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey(); String line = c.element().getValue(); for (String word : line.split("\\W+")) { // Log INFO messages when the word “love” is found. if (word.toLowerCase().equals("love")) { LOG.info("Found {}", word.toLowerCase()); } if (!word.isEmpty()) { c.output(KV.of(uri, word.toLowerCase())); } } } })); // Compute a mapping from each word to the total // number of documents in which it appears. PCollection<KV<String, Long>> wordToDocCount = uriToWords .apply(RemoveDuplicates.<KV<URI, String>>create().setName("RemoveDuplicateWords")) .apply(Values.<String>create()) .apply(Count.<String>perElement().setName("CountDocs")); // Compute a mapping from each URI to the total // number of words in the document associated with that URI. PCollection<KV<URI, Long>> uriToWordTotal = uriToWords .apply(Keys.<URI>create().setName("GetURIs2")) .apply(Count.<URI>perElement().setName("CountWords")); // Count, for each (URI, word) pair, the number of // occurrences of that word in the document associated // with the URI. PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords.apply(Count.<KV<URI, String>>perElement().setName("CountWordDocPairs")); // Adjust the above collection to a mapping from // (URI, word) pairs to counts into an isomorphic mapping // from URI to (word, count) pairs, to prepare for a join // by the URI key. PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount.apply( ParDo.named("ShiftKeys") .of( new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() { private static final long serialVersionUID = 0; @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey().getKey(); String word = c.element().getKey().getValue(); Long occurrences = c.element().getValue(); c.output(KV.of(uri, KV.of(word, occurrences))); } })); // Prepare to join the mapping of URI to (word, count) pairs with // the mapping of URI to total word counts, by associating // each of the input PCollection<KV<URI, ...>> with // a tuple tag. Each input must have the same key type, URI // in this case. The type parameter of the tuple tag matches // the types of the values for each collection. final TupleTag<Long> wordTotalsTag = new TupleTag<Long>(); final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<KV<String, Long>>(); KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple.of(wordTotalsTag, uriToWordTotal) .and(wordCountsTag, uriToWordAndCount); // Perform a CoGroupByKey (a sort of pre-join) on the prepared // inputs. This yields a mapping from URI to a CoGbkResult // (CoGroupByKey Result). The CoGbkResult is a mapping // from the above tuple tags to the values in each input // associated with a particular URI. In this case, each // KV<URI, CoGbkResult> group a URI with the total number of // words in that document as well as all the (word, count) // pairs for particular words. PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput.apply(CoGroupByKey.<URI>create().setName("CoGroupByURI")); // Compute a mapping from each word to a (URI, term frequency) // pair for each URI. A word's term frequency for a document // is simply the number of times that word occurs in the document // divided by the total number of words in the document. PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal.apply( ParDo.named("ComputeTermFrequencies") .of( new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() { private static final long serialVersionUID = 0; @Override public void processElement(ProcessContext c) { URI uri = c.element().getKey(); Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); for (KV<String, Long> wordAndCount : c.element().getValue().getAll(wordCountsTag)) { String word = wordAndCount.getKey(); Long wordCount = wordAndCount.getValue(); Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); c.output(KV.of(word, KV.of(uri, termFrequency))); } } })); // Compute a mapping from each word to its document frequency. // A word's document frequency in a corpus is the number of // documents in which the word appears divided by the total // number of documents in the corpus. Note how the total number of // documents is passed as a side input; the same value is // presented to each invocation of the DoFn. PCollection<KV<String, Double>> wordToDf = wordToDocCount.apply( ParDo.named("ComputeDocFrequencies") .withSideInputs(totalDocuments) .of( new DoFn<KV<String, Long>, KV<String, Double>>() { private static final long serialVersionUID = 0; @Override public void processElement(ProcessContext c) { String word = c.element().getKey(); Long documentCount = c.element().getValue(); Long documentTotal = c.sideInput(totalDocuments); Double documentFrequency = documentCount.doubleValue() / documentTotal.doubleValue(); c.output(KV.of(word, documentFrequency)); } })); // Join the term frequency and document frequency // collections, each keyed on the word. final TupleTag<KV<URI, Double>> tfTag = new TupleTag<KV<URI, Double>>(); final TupleTag<Double> dfTag = new TupleTag<Double>(); PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple.of(tfTag, wordToUriAndTf) .and(dfTag, wordToDf) .apply(CoGroupByKey.<String>create()); // Compute a mapping from each word to a (URI, TF-IDF) score // for each URI. There are a variety of definitions of TF-IDF // ("term frequency - inverse document frequency") score; // here we use a basic version that is the term frequency // divided by the log of the document frequency. PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = wordToUriAndTfAndDf.apply( ParDo.named("ComputeTfIdf") .of( new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() { private static final long serialVersionUID = 0; @Override public void processElement(ProcessContext c) { String word = c.element().getKey(); Double df = c.element().getValue().getOnly(dfTag); for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) { URI uri = uriAndTf.getKey(); Double tf = uriAndTf.getValue(); Double tfIdf = tf * Math.log(1 / df); c.output(KV.of(word, KV.of(uri, tfIdf))); } } })); return wordToUriAndTfIdf; }