Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.beam.sdk.loadtests;

/** Writes {@link LoadTestResult} to console. */
public class ConsoleResultPublisher {

static void publish(LoadTestResult result) {
System.out.println(String.format("Total bytes: %s", result.getTotalBytesCount()));
System.out.println(String.format("Total time (millis): %s", result.getRuntime()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,17 @@
package org.apache.beam.sdk.loadtests;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import java.util.Optional;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
import org.apache.beam.sdk.io.synthetic.SyntheticOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticStep;
import org.apache.beam.sdk.loadtests.metrics.MetricsPublisher;
import org.apache.beam.sdk.loadtests.metrics.TimeMonitor;
import org.apache.beam.sdk.testutils.publishing.BigQueryResultsPublisher;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
Expand Down Expand Up @@ -62,18 +64,51 @@ abstract class LoadTest<OptionsT extends LoadTestOptions> {
/** The load test pipeline implementation. */
abstract void loadTest() throws IOException;

/** Runs the load test. */
/**
* Runs the load test, collects and publishes test results to various data store and/or console.
*/
public PipelineResult run() throws IOException {
long testStartTime = System.currentTimeMillis();

loadTest();

PipelineResult result = pipeline.run();
result.waitUntilFinish();

MetricsPublisher.toConsole(result, metricsNamespace);
LoadTestResult testResult = LoadTestResult.create(result, metricsNamespace, testStartTime);

ConsoleResultPublisher.publish(testResult);

if (options.getPublishToBigQuery()) {
publishResultToBigQuery(testResult);
}
return result;
}

private void publishResultToBigQuery(LoadTestResult testResult) {
String dataset = options.getBigQueryDataset();
String table = options.getBigQueryTable();
checkBigQueryOptions(dataset, table);

ImmutableMap<String, String> schema =
ImmutableMap.<String, String>builder()
.put("timestamp", "timestamp")
.put("runtime", "float")
.put("total_bytes_count", "integer")
.build();

BigQueryResultsPublisher.create(dataset, schema).publish(testResult, table);
}

private static void checkBigQueryOptions(String dataset, String table) {
Preconditions.checkArgument(
dataset != null,
"Please specify --bigQueryDataset option if you want to publish to BigQuery");

Preconditions.checkArgument(
table != null, "Please specify --bigQueryTable option if you want to publish to BigQuery");
}

<T extends SyntheticOptions> T fromJsonString(String json, Class<T> type) throws IOException {
ObjectMapper mapper = new ObjectMapper();
T result = mapper.readValue(json, type);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.sdk.loadtests;

import org.apache.beam.sdk.options.ApplicationNameOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
Expand All @@ -37,6 +38,22 @@ public interface LoadTestOptions extends PipelineOptions, ApplicationNameOptions

void setStepOptions(String stepOptions);

@Description("Whether the results should be published to BigQuery database")
@Default.Boolean(false)
Boolean getPublishToBigQuery();

void setPublishToBigQuery(Boolean publishToBigQuery);

@Description("BigQuery dataset name")
String getBigQueryDataset();

void setBigQueryDataset(String dataset);

@Description("BigQuery table name")
String getBigQueryTable();

void setBigQueryTable(String tableName);

static <T extends LoadTestOptions> T readFromArgs(String[] args, Class<T> optionsClass) {
return PipelineOptionsFactory.fromArgs(args).withValidation().as(optionsClass);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.beam.sdk.loadtests;

import com.google.common.collect.ImmutableMap;
import java.util.Map;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.testutils.TestResult;
import org.apache.beam.sdk.testutils.metrics.MetricsReader;

/** POJO that represents load test results. */
public class LoadTestResult implements TestResult {

private final Long timestamp;

private final Long runtime;

private final Long totalBytesCount;

private LoadTestResult(Long timestamp, Long runtime, Long totalBytesCount) {
this.timestamp = timestamp;
this.runtime = runtime;
this.totalBytesCount = totalBytesCount;
}

/** Constructs {@link LoadTestResult} from {@link PipelineResult}. */
static LoadTestResult create(PipelineResult result, String namespace, long now) {
MetricsReader reader = new MetricsReader(result, namespace);

return new LoadTestResult(
now,
reader.getEndTimeMetric("runtime") - reader.getStartTimeMetric("runtime"),
reader.getCounterMetric("totalBytes.count"));
}

public Long getRuntime() {
return runtime;
}

public Long getTotalBytesCount() {
return totalBytesCount;
}

@Override
public Map<String, Object> toMap() {
return ImmutableMap.<String, Object>builder()
.put("timestamp", timestamp)
.put("runtime", runtime)
.put("totalBytesCount", totalBytesCount)
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,16 @@

import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.testutils.metrics.MetricsReader;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.KV;

/**
* Monitor that records the number of bytes flowing through a PCollection.
*
* <p>To use: apply a monitor in a desired place in the pipeline. This will capture how many bytes
* flew through this DoFn which then can be collected and written out using {@link
* MetricsPublisher}.
* flew through this DoFn. Such information can be then collected and written out and queried using
* {@link MetricsReader}.
*/
public class ByteMonitor extends DoFn<KV<byte[], byte[]>, KV<byte[], byte[]>> {

Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,16 @@

import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.testutils.metrics.MetricsReader;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.KV;

/**
* Monitor that records processing time distribution in the pipeline.
*
* <p>To use: apply a monitor directly after each source and sink transform. This will capture a
* distribution of element processing timestamps, which can be collected and written out using
* {@link MetricsPublisher}.
* distribution of element processing timestamps, which can be collected and queried using {@link
* MetricsReader}.
*/
public class TimeMonitor<K, V> extends DoFn<KV<K, V>, KV<K, V>> {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@
import org.apache.beam.sdk.nexmark.model.Bid;
import org.apache.beam.sdk.nexmark.model.Person;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testutils.publishing.BigQueryClient;
import org.apache.beam.sdk.testutils.publishing.BigQueryResultsPublisher;
import org.joda.time.Duration;
import org.joda.time.Instant;

Expand Down Expand Up @@ -142,8 +142,19 @@ void runAll(String[] args) throws IOException {
}

if (options.getExportSummaryToBigQuery()) {
BigQueryClient publisher = BigQueryClient.create(options.getBigQueryDataset());
savePerfsToBigQuery(publisher, options, actual, start);
ImmutableMap<String, String> schema =
ImmutableMap.<String, String>builder()
.put("timestamp", "timestamp")
.put("runtimeSec", "float")
.put("eventsPerSec", "float")
.put("numResults", "integer")
.build();

savePerfsToBigQuery(
BigQueryResultsPublisher.create(options.getBigQueryDataset(), schema),
options,
actual,
start);
}
} finally {
if (options.getMonitorJobs()) {
Expand All @@ -161,7 +172,7 @@ void runAll(String[] args) throws IOException {

@VisibleForTesting
static void savePerfsToBigQuery(
BigQueryClient bigQueryClient,
BigQueryResultsPublisher publisher,
NexmarkOptions options,
Map<NexmarkConfiguration, NexmarkPerf> perfs,
Instant start) {
Expand All @@ -172,25 +183,7 @@ static void savePerfsToBigQuery(
options.getQueryLanguage(), entry.getKey().query.getNumberOrName());
String tableName = NexmarkUtils.tableName(options, queryName, 0L, null);

ImmutableMap<String, String> schema =
ImmutableMap.<String, String>builder()
.put("timestamp", "timestamp")
.put("runtimeSec", "float")
.put("eventsPerSec", "float")
.put("numResults", "integer")
.build();
bigQueryClient.createTableIfNotExists(tableName, schema);

// convert millis to seconds (it's a BigQuery's requirement).
Map<String, Object> record =
ImmutableMap.<String, Object>builder()
.put("timestamp", start.getMillis() / 1000)
.put("runtimeSec", entry.getValue().runtimeSec)
.put("eventsPerSec", entry.getValue().eventsPerSec)
.put("numResults", entry.getValue().numResults)
.build();

bigQueryClient.insertRow(record, tableName);
publisher.publish(entry.getValue(), tableName, start.getMillis());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,16 @@

import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.sdk.testutils.TestResult;

/** Summary of performance for a particular run of a configuration. */
public class NexmarkPerf {
public class NexmarkPerf implements TestResult {

/**
* A sample of the number of events and number of results (if known) generated at a particular
* time.
Expand Down Expand Up @@ -152,4 +156,9 @@ public boolean anyActivity(NexmarkPerf that) {
}
return false;
}

@Override
public Map<String, Object> toMap() {
return NexmarkUtils.MAPPER.convertValue(this, new TypeReference<Map<String, Object>>() {});
}
}
Loading