You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@nemo.apache.org by GitBox <gi...@apache.org> on 2022/08/01 03:40:41 UTC

[GitHub] [incubator-nemo] jeongyooneo commented on a diff in pull request #314: [NEMO-481] Add Stream examples

jeongyooneo commented on code in PR #314:
URL: https://github.com/apache/incubator-nemo/pull/314#discussion_r934103642


##########
examples/beam/src/main/java/org/apache/nemo/examples/beam/EDGARDocumentSuccessRate.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.nemo.examples.beam;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Application for EDGAR dataset.
+ * Format: ip, date, time, zone, doc_cik, access number, doc_name, code, size, idx, norefer, noagent, find, crawler.
+ * Calculate the success rate of each document.
+ */
+public final class EDGARDocumentSuccessRate {
+  private static final Logger LOG = LoggerFactory.getLogger(EDGARDocumentSuccessRate.class.getName());
+
+  /**
+   * Private Constructor.
+   */
+  private EDGARDocumentSuccessRate() {
+  }
+
+  /**
+   * Main function for the BEAM program.
+   *
+   * @param args arguments.
+   */
+  public static void main(final String[] args) {
+    final String inputFilePath = args[0];
+    final String windowType = args[1];
+    final String outputFilePath = args[2];
+
+    final Window<KV<String, Integer>> windowFn;
+    if (windowType.equals("fixed")) {
+      windowFn = Window.into(FixedWindows.of(Duration.standardSeconds(5)));
+    } else {
+      windowFn = Window.into(SlidingWindows.of(Duration.standardSeconds(10))
+        .every(Duration.standardSeconds(5)));
+    }
+
+    final PipelineOptions options = NemoPipelineOptionsFactory.create();
+    options.setJobName("EDGAR: Document retrieval success rate");
+
+    final Pipeline p = Pipeline.create(options);
+
+    final PCollection<KV<String, Integer>> source = GenericSourceSink.read(p, inputFilePath)
+      .apply(ParDo.of(new DoFn<String, KV<String, Integer>>() {
+        @ProcessElement
+        public void processElement(@DoFn.Element final String elem,
+                                   final OutputReceiver<KV<String, Integer>> out) {
+          final String[] splitt = elem.split(",");
+          final Integer success = splitt[7].startsWith("2") ? 1 : 0;
+          try {
+            out.outputWithTimestamp(KV.of(splitt[6], success), Instant.parse(splitt[1] + "T" + splitt[2] + "Z"));
+          } catch (Exception e) {
+            LOG.warn("Parsing failed due to: ", e);
+          }
+        }
+      }));
+    source.apply(windowFn)

Review Comment:
   Could you elaborate on how does L86-L94 calculate the success rate of document retrieval?
   As far as I understood, L72-L85 records the number of successes and its corresponding timestamp, and L86-L94 is where the success rate is calculated. If that is so, could you explain how L86-L94 calculates the success rate?



##########
examples/beam/src/main/java/org/apache/nemo/examples/beam/WindowedWordCount.java:
##########
@@ -43,36 +45,45 @@ private WindowedWordCount() {
   public static final String INPUT_TYPE_BOUNDED = "bounded";
   public static final String INPUT_TYPE_UNBOUNDED = "unbounded";
   private static final String SPLITTER = "!";
-
+  public static final Random RAND = new Random();
 
   /**
    * @param p    pipeline.
-   * @param args arguments.
+   * @param inputType input type arg.
+   * @param inputFilePath input file path arg.
    * @return source.
    */
   private static PCollection<KV<String, Long>> getSource(
     final Pipeline p,
-    final String[] args) {
+    final String inputType,
+    final String inputFilePath) {
 
-    final String inputType = args[2];
     if (inputType.compareTo(INPUT_TYPE_BOUNDED) == 0) {
-      final String inputFilePath = args[3];
       return GenericSourceSink.read(p, inputFilePath)
         .apply(ParDo.of(new DoFn<String, String>() {
           @ProcessElement
           public void processElement(@Element final String elem,
                                      final OutputReceiver<String> out) {
             final String[] splitt = elem.split(SPLITTER);
-            out.outputWithTimestamp(splitt[0], new Instant(Long.valueOf(splitt[1])));
+            if (splitt.length > 1 && splitt[1].matches("[0-9]+")) {
+              out.outputWithTimestamp(splitt[0], new Instant(Long.valueOf(splitt[1])));
+            } else {
+              final long timestamp = System.currentTimeMillis() - RAND.nextInt(1000000);
+              out.outputWithTimestamp(elem, new Instant(timestamp));

Review Comment:
   Is it okay for the correctness of end-to-end operation if we assign dummy timestamp for documents with format error in the `date` field?



##########
examples/beam/src/main/java/org/apache/nemo/examples/beam/EDGARTop10BadRefererDocs.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.nemo.examples.beam;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Application for EDGAR dataset.
+ * Format: ip, date, time, zone, doc_cik, access number, doc_name, code, size, idx, norefer, noagent, find, crawler.
+ * Top 10 documents tha produce the most bad referer errors.

Review Comment:
   - tha -> that
   - the most bad -> the worst
   



##########
examples/beam/src/main/java/org/apache/nemo/examples/beam/EDGARTop10BadRefererDocs.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.nemo.examples.beam;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Application for EDGAR dataset.
+ * Format: ip, date, time, zone, doc_cik, access number, doc_name, code, size, idx, norefer, noagent, find, crawler.
+ * Top 10 documents tha produce the most bad referer errors.
+ */
+public final class EDGARTop10BadRefererDocs {
+  private static final Logger LOG = LoggerFactory.getLogger(EDGARTop10BadRefererDocs.class.getName());
+
+  /**
+   * Private Constructor.
+   */
+  private EDGARTop10BadRefererDocs() {
+  }
+
+  /**
+   * Main function for the BEAM program.
+   *
+   * @param args arguments.
+   */
+  public static void main(final String[] args) {
+    final String inputFilePath = args[0];
+    final String windowType = args[1];
+    final String outputFilePath = args[2];
+
+    final Window<KV<Object, Integer>> windowFn;
+    if (windowType.equals("fixed")) {
+      windowFn = Window.into(FixedWindows.of(Duration.standardSeconds(5)));
+    } else {
+      windowFn = Window.into(SlidingWindows.of(Duration.standardSeconds(10))
+        .every(Duration.standardSeconds(5)));
+    }
+
+    final PipelineOptions options = NemoPipelineOptionsFactory.create();
+    options.setJobName("EDGAR: Top 10 documents with bad referers");
+
+    final Pipeline p = Pipeline.create(options);
+
+    final PCollection<KV<Object, Integer>> source = GenericSourceSink.read(p, inputFilePath)
+      .apply(ParDo.of(new DoFn<String, KV<Object, Integer>>() {
+        @ProcessElement
+        public void processElement(@DoFn.Element final String elem,
+                                   final OutputReceiver<KV<Object, Integer>> out) {
+          final String[] splitt = elem.split(",");
+          final Integer failure = splitt[7].startsWith("2") ? 0 : 1;
+          try {
+            out.outputWithTimestamp(KV.of(splitt[6], failure), Instant.parse(splitt[1] + "T" + splitt[2] + "Z"));
+          } catch (Exception e) {
+            LOG.warn("Parsing failed due to: ", e);
+          }
+        }
+      }));
+    source.setCoder(KvCoder.of(ObjectCoderForString.of(), VarIntCoder.of()));
+    source.apply(windowFn)
+      .apply(Mean.perKey())

Review Comment:
   This question is simply for understanding the operation of `EDGARTop10BadRefererDocs` better:
   Does the 'top 10 worst referer docs' mean 'top 10 doc_names with largest average number of failure counts'?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@nemo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org