Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 31429200CFC for ; Wed, 23 Aug 2017 19:09:18 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2FD7B1672BB; Wed, 23 Aug 2017 17:09:18 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D1F76161C3A for ; Wed, 23 Aug 2017 19:09:16 +0200 (CEST) Received: (qmail 27561 invoked by uid 500); 23 Aug 2017 17:09:15 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 23983 invoked by uid 99); 23 Aug 2017 17:09:12 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 23 Aug 2017 17:09:12 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 97BDCF5F0A; Wed, 23 Aug 2017 17:09:09 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: iemejia@apache.org To: commits@beam.apache.org Date: Wed, 23 Aug 2017 17:09:27 -0000 Message-Id: <7d4edb84ca2a4b7d9d866838f4b8c7d7@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [19/55] [abbrv] beam git commit: Activate monitoring on NexmarkSparkRunner and on specific runners archived-at: Wed, 23 Aug 2017 17:09:18 -0000 Activate monitoring on NexmarkSparkRunner and on specific runners issue #28 Fix compilation issue after rebase + make checkstyle happy again Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a1fe33bc Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a1fe33bc Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a1fe33bc Branch: refs/heads/master Commit: a1fe33bc122b26960697c87620ca0dc2ed522e56 Parents: a095e40 Author: Etienne Chauchot Authored: Wed Mar 15 15:25:41 2017 +0100 Committer: Ismaël Mejía Committed: Wed Aug 23 19:07:27 2017 +0200 ---------------------------------------------------------------------- integration/java/nexmark/pom.xml | 32 ++++++++++---------- .../integration/nexmark/NexmarkApexRunner.java | 2 -- .../nexmark/NexmarkDirectRunner.java | 5 --- .../integration/nexmark/NexmarkFlinkRunner.java | 12 +------- .../nexmark/NexmarkGoogleDriver.java | 2 -- .../nexmark/NexmarkGoogleRunner.java | 2 +- .../beam/integration/nexmark/NexmarkRunner.java | 13 ++++---- .../integration/nexmark/NexmarkSparkDriver.java | 4 +-- .../integration/nexmark/NexmarkSparkRunner.java | 11 +------ .../beam/integration/nexmark/NexmarkUtils.java | 3 +- .../apache/beam/integration/nexmark/Query5.java | 3 +- 11 files changed, 31 insertions(+), 58 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/pom.xml ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 07d14c2..febd96d 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -179,28 +179,28 @@ beam-runners-flink_2.10 - - - - - - + + org.apache.flink + flink-shaded-hadoop2 + ${flink.version} + provided + org.apache.beam beam-runners-spark - - - - - - - - - - + + org.apache.spark + spark-core_2.10 + ${spark.version} + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java index f2da1c7..ea46082 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java @@ -18,8 +18,6 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; -import org.apache.beam.runners.apex.ApexRunnerResult; -import org.apache.beam.sdk.PipelineResult; /** * Run a query using the Apex runner. http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java index ee234b1..c70e41e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java @@ -17,11 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.direct.DirectRunner; -import org.apache.beam.sdk.PipelineResult; - /** * Run a single query using the Direct Runner. */ http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java index a8b4401..8e22917 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -17,10 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; -import org.apache.beam.runners.flink.FlinkRunnerResult; -import org.apache.beam.sdk.PipelineResult; - /** * Run a query using the Flink runner. */ @@ -42,7 +38,7 @@ public class NexmarkFlinkRunner extends NexmarkRunner { * Find a 'steady state' events/sec from {@code snapshots} and * store it in {@code perf} if found. */ - protected void captureSteadyState(NexmarkPerf perf, List snapshots) { + protected void captureSteadyState(NexmarkPerf perf, + List snapshots) { if (!options.isStreaming()) { return; } @@ -365,7 +364,9 @@ public abstract class NexmarkRunner { return perf; } - String getJobId(PipelineResult job){return "";} + String getJobId(PipelineResult job) { + return ""; + } // TODO specific to dataflow, see if we can find an equivalent /* @@ -926,8 +927,8 @@ public abstract class NexmarkRunner { new TableFieldSchema().setName("index").setType("INTEGER"), new TableFieldSchema().setName("value").setType("STRING"))))); NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec); - BigQueryIO.Write.Bound io = - BigQueryIO.Write.to(tableSpec) + BigQueryIO.Write io = + BigQueryIO.write().to(tableSpec) .withSchema(tableSchema) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND); http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java index 1ea963d..a46d38a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark; import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.options.PipelineOptionsFactory; /** @@ -39,7 +38,8 @@ class NexmarkSparkDriver extends NexmarkDriver runner = p.getRunner(); //TODO Ismael check +// PipelineRunner runner = p.getRunner(); // if (runner instanceof DirectRunner) { // // Disable randomization of output since we want to check batch and streaming match the // // model both locally and on the cloud. http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java ---------------------------------------------------------------------- diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java index 7001986..9020494 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java @@ -67,7 +67,8 @@ class Query5 extends NexmarkQuery { // Count the number of bids per auction id. .apply(Count.perElement()) - // We'll want to keep all auctions with the maximal number of bids. + //TODO replace by simple key + // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. .apply(name + ".ToSingletons", ParDo.of(new DoFn, KV, Long>>() {