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
5 changes: 5 additions & 0 deletions ingestion/src/main/java/feast/ingestion/ImportJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import feast.ingestion.transform.ValidateFeatureRows;
import feast.ingestion.transform.WriteFailedElementToBigQuery;
import feast.ingestion.transform.metrics.WriteFailureMetricsTransform;
import feast.ingestion.transform.metrics.WriteInflightMetricsTransform;
import feast.ingestion.transform.metrics.WriteSuccessMetricsTransform;
import feast.ingestion.utils.ResourceUtil;
import feast.ingestion.utils.SpecUtil;
Expand Down Expand Up @@ -135,6 +136,10 @@ public static PipelineResult runPipeline(ImportOptions options) throws IOExcepti
.setFailureTag(DEADLETTER_OUT)
.build());

validatedRows
.get(FEATURE_ROW_OUT)
.apply("WriteInflightMetrics", WriteInflightMetricsTransform.create(store.getName()));

// Step 3. Write FeatureRow to the corresponding Store.
WriteResult writeFeatureRows =
validatedRows.get(FEATURE_ROW_OUT).apply("WriteFeatureRowToStore", featureSink.write());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package feast.ingestion.transform.metrics;

import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.METRIC_NAMESPACE_TAG_KEY;

import com.google.auto.value.AutoValue;
import com.timgroup.statsd.NonBlockingStatsDClient;
import com.timgroup.statsd.StatsDClient;
Expand All @@ -39,6 +41,8 @@ public abstract class WriteDeadletterRowMetricsDoFn extends DoFn<FailedElement,

public abstract String getStoreName();

public abstract String getMetricNamespace();

public abstract String getStatsdHost();

public abstract int getStatsdPort();
Expand All @@ -54,6 +58,8 @@ public abstract static class Builder {

public abstract Builder setStoreName(String storeName);

public abstract Builder setMetricNamespace(String metricNamespace);

public abstract Builder setStatsdHost(String statsdHost);

public abstract Builder setStatsdPort(int statsdPort);
Expand All @@ -74,6 +80,7 @@ public void processElement(ProcessContext c) {
"deadletter_row_count",
1,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
PROJECT_TAG_KEY + ":" + ignored.getProjectName(),
FEATURE_SET_NAME_TAG_KEY + ":" + ignored.getFeatureSetName(),
FEATURE_SET_VERSION_TAG_KEY + ":" + ignored.getFeatureSetVersion(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
public abstract class WriteFailureMetricsTransform
extends PTransform<PCollection<FailedElement>, PDone> {

private static final String METRIC_NAMESPACE = "WriteToStoreFailure";

public abstract String getStoreName();

public static WriteFailureMetricsTransform create(String storeName) {
Expand All @@ -42,6 +44,7 @@ public PDone expand(PCollection<FailedElement> input) {
"WriteDeadletterMetrics",
ParDo.of(
WriteDeadletterRowMetricsDoFn.newBuilder()
.setMetricNamespace(METRIC_NAMESPACE)
.setStatsdHost(options.getStatsdHost())
.setStatsdPort(options.getStatsdPort())
.setStoreName(getStoreName())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,7 @@
*/
package feast.ingestion.transform.metrics;

import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.FEATURE_SET_NAME_TAG_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.FEATURE_SET_PROJECT_TAG_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.FEATURE_SET_VERSION_TAG_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.FEATURE_TAG_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.INGESTION_JOB_NAME_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.METRIC_PREFIX;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.STORE_TAG_KEY;
import static feast.ingestion.transform.metrics.WriteRowMetricsDoFn.*;

import com.google.auto.value.AutoValue;
import com.timgroup.statsd.NonBlockingStatsDClient;
Expand Down Expand Up @@ -64,6 +58,8 @@ public abstract class WriteFeatureValueMetricsDoFn

abstract String getStoreName();

abstract String getMetricNamespace();

abstract String getStatsdHost();

abstract int getStatsdPort();
Expand All @@ -77,6 +73,8 @@ abstract static class Builder {

abstract Builder setStoreName(String storeName);

abstract Builder setMetricNamespace(String metricNamespace);

abstract Builder setStatsdHost(String statsdHost);

abstract Builder setStatsdPort(int statsdPort);
Expand Down Expand Up @@ -161,6 +159,7 @@ public void processElement(
DoubleSummaryStatistics stats = entry.getValue();
String[] tags = {
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + projectName,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + version,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* SPDX-License-Identifier: Apache-2.0
* Copyright 2018-2019 The Feast Authors
*
* Licensed 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
*
* https://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 feast.ingestion.transform.metrics;

import com.google.auto.value.AutoValue;
import feast.ingestion.options.ImportOptions;
import feast.types.FeatureRowProto.FeatureRow;
import org.apache.beam.sdk.transforms.*;
import org.apache.beam.sdk.transforms.windowing.FixedWindows;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
import org.joda.time.Duration;

@AutoValue
public abstract class WriteInflightMetricsTransform
extends PTransform<PCollection<FeatureRow>, PDone> {

public static final String METRIC_NAMESPACE = "Inflight";

public abstract String getStoreName();

public static WriteInflightMetricsTransform create(String storeName) {
return new AutoValue_WriteInflightMetricsTransform(storeName);
}

@Override
public PDone expand(PCollection<FeatureRow> input) {
ImportOptions options = input.getPipeline().getOptions().as(ImportOptions.class);

switch (options.getMetricsExporterType()) {
case "statsd":
input.apply(
"WriteRowMetrics",
ParDo.of(
WriteRowMetricsDoFn.newBuilder()
.setMetricNamespace(METRIC_NAMESPACE)
.setStatsdHost(options.getStatsdHost())
.setStatsdPort(options.getStatsdPort())
.setStoreName(getStoreName())
.build()));

// 1. Apply a fixed window
// 2. Group feature row by feature set reference
// 3. Calculate min, max, mean, percentiles of numerical values of features in the window
// and
// 4. Send the aggregate value to StatsD metric collector.
//
// NOTE: window is applied here so the metric collector will not be overwhelmed with
// metrics data. And for metric data, only statistic of the values are usually required
// vs the actual values.
input
.apply(
"FixedWindow",
Window.into(
FixedWindows.of(
Duration.standardSeconds(
options.getWindowSizeInSecForFeatureValueMetric()))))
.apply(
"ConvertTo_FeatureSetRefToFeatureRow",
ParDo.of(
new DoFn<FeatureRow, KV<String, FeatureRow>>() {
@ProcessElement
public void processElement(ProcessContext c, @Element FeatureRow featureRow) {
c.output(KV.of(featureRow.getFeatureSet(), featureRow));
}
}))
.apply("GroupByFeatureSetRef", GroupByKey.create())
.apply(
"WriteFeatureValueMetrics",
ParDo.of(
WriteFeatureValueMetricsDoFn.newBuilder()
.setMetricNamespace(METRIC_NAMESPACE)
.setStatsdHost(options.getStatsdHost())
.setStatsdPort(options.getStatsdPort())
.setStoreName(getStoreName())
.build()));

return PDone.in(input.getPipeline());
case "none":
default:
input.apply(
"Noop",
ParDo.of(
new DoFn<FeatureRow, Void>() {
@ProcessElement
public void processElement(ProcessContext c) {}
}));
return PDone.in(input.getPipeline());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ public abstract class WriteRowMetricsDoFn extends DoFn<FeatureRow, Void> {
private static final Logger log = org.slf4j.LoggerFactory.getLogger(WriteRowMetricsDoFn.class);

public static final String METRIC_PREFIX = "feast_ingestion";
public static final String METRIC_NAMESPACE_TAG_KEY = "feast_metric_namespace";
public static final String STORE_TAG_KEY = "feast_store";
public static final String FEATURE_SET_PROJECT_TAG_KEY = "feast_project_name";
public static final String FEATURE_SET_NAME_TAG_KEY = "feast_featureSet_name";
Expand All @@ -41,6 +42,8 @@ public abstract class WriteRowMetricsDoFn extends DoFn<FeatureRow, Void> {

public abstract String getStoreName();

public abstract String getMetricNamespace();

public abstract String getStatsdHost();

public abstract int getStatsdPort();
Expand All @@ -65,6 +68,8 @@ public abstract static class Builder {

public abstract Builder setStoreName(String storeName);

public abstract Builder setMetricNamespace(String metricNamespace);

public abstract Builder setStatsdHost(String statsdHost);

public abstract Builder setStatsdPort(int statsdPort);
Expand Down Expand Up @@ -93,6 +98,7 @@ public void processElement(ProcessContext c) {
"feature_row_lag_ms",
System.currentTimeMillis() - eventTimestamp,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + featureSetProject,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + featureSetVersion,
Expand All @@ -102,6 +108,7 @@ public void processElement(ProcessContext c) {
"feature_row_event_time_epoch_ms",
eventTimestamp,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + featureSetProject,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + featureSetVersion,
Expand All @@ -113,6 +120,7 @@ public void processElement(ProcessContext c) {
"feature_value_lag_ms",
System.currentTimeMillis() - eventTimestamp,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + featureSetProject,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + featureSetVersion,
Expand All @@ -123,6 +131,7 @@ public void processElement(ProcessContext c) {
"feature_value_missing_count",
1,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + featureSetProject,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + featureSetVersion,
Expand All @@ -135,6 +144,7 @@ public void processElement(ProcessContext c) {
"feature_row_ingested_count",
1,
STORE_TAG_KEY + ":" + getStoreName(),
METRIC_NAMESPACE_TAG_KEY + ":" + getMetricNamespace(),
FEATURE_SET_PROJECT_TAG_KEY + ":" + featureSetProject,
FEATURE_SET_NAME_TAG_KEY + ":" + featureSetName,
FEATURE_SET_VERSION_TAG_KEY + ":" + featureSetVersion,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ public PDone expand(PCollection<FeatureRow> input) {
"WriteRowMetrics",
ParDo.of(
WriteRowMetricsDoFn.newBuilder()
.setMetricNamespace(METRIC_NAMESPACE)
.setStatsdHost(options.getStatsdHost())
.setStatsdPort(options.getStatsdPort())
.setStoreName(getStoreName())
Expand Down Expand Up @@ -99,6 +100,7 @@ public void processElement(ProcessContext c, @Element FeatureRow featureRow) {
"WriteFeatureValueMetrics",
ParDo.of(
WriteFeatureValueMetricsDoFn.newBuilder()
.setMetricNamespace(METRIC_NAMESPACE)
.setStatsdHost(options.getStatsdHost())
.setStatsdPort(options.getStatsdPort())
.setStoreName(getStoreName())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ public void shouldSendCorrectStatsDMetrics() throws IOException, InterruptedExce
.apply(
ParDo.of(
WriteFeatureValueMetricsDoFn.newBuilder()
.setMetricNamespace("test")
.setStatsdHost("localhost")
.setStatsdPort(STATSD_SERVER_PORT)
.setStoreName("store")
Expand Down
Loading