Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add Lineage metrics to FileSystems #32090

Merged
merged 3 commits into from
Aug 14, 2024
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
Expand Up @@ -687,6 +687,7 @@ protected final List<KV<FileResult<DestinationT>, ResourceId>> finalizeDestinati
distinctFilenames.get(finalFilename));
distinctFilenames.put(finalFilename, result);
outputFilenames.add(KV.of(result, finalFilename));
FileSystems.reportSinkLineage(finalFilename);
}
return outputFilenames;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,7 @@ public final List<? extends FileBasedSource<T>> split(
splitResults.size());
return splitResults;
} else {
FileSystems.reportSourceLineage(getSingleFileMetadata().resourceId());
if (isSplittable()) {
@SuppressWarnings("unchecked")
List<FileBasedSource<T>> splits =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.metrics.Lineage;

/**
* File system interface in Beam.
Expand Down Expand Up @@ -155,4 +156,11 @@ protected abstract void rename(
* @see <a href="https://www.ietf.org/rfc/rfc2396.txt">RFC 2396</a>
*/
protected abstract String getScheme();

/**
* Report {@link Lineage} metrics for resource id.
*
* <p>Unless override by FileSystem implementations, default to no-op.
*/
protected void reportLineage(ResourceIdT unusedId, Lineage unusedLineage) {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.beam.sdk.io.fs.MoveOptions.StandardMoveOptions;
import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.util.common.ReflectHelpers;
import org.apache.beam.sdk.values.KV;
Expand Down Expand Up @@ -395,6 +396,16 @@ public ResourceId apply(@Nonnull Metadata input) {
.delete(resourceIdsToDelete);
}

/** Report source {@link Lineage} metrics for resource id. */
public static void reportSourceLineage(ResourceId resourceId) {
getFileSystemInternal(resourceId.getScheme()).reportLineage(resourceId, Lineage.getSources());
}

/** Report sink {@link Lineage} metrics for resource id. */
public static void reportSinkLineage(ResourceId resourceId) {
getFileSystemInternal(resourceId.getScheme()).reportLineage(resourceId, Lineage.getSinks());
}

private static class FilterResult {
public List<ResourceId> resultSources = new ArrayList();
public List<ResourceId> resultDestinations = new ArrayList();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.io.IOException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
Expand Down Expand Up @@ -123,8 +124,9 @@ protected abstract T makeOutput(
public void process(ProcessContext c) throws IOException {
FileIO.ReadableFile file = c.element().getKey();
OffsetRange range = c.element().getValue();
ResourceId resourceId = file.getMetadata().resourceId();
FileBasedSource<InT> source =
CompressedSource.from(createSource.apply(file.getMetadata().resourceId().toString()))
CompressedSource.from(createSource.apply(resourceId.toString()))
.withCompression(file.getCompression());
try (BoundedSource.BoundedReader<InT> reader =
source
Expand All @@ -138,6 +140,7 @@ public void process(ProcessContext c) throws IOException {
throw e;
}
}
FileSystems.reportSourceLineage(resourceId);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,26 +19,109 @@

import java.util.HashSet;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
* Standard collection of metrics used to record source and sinks information for lineage tracking.
*/
public class Lineage {

public static final String LINEAGE_NAMESPACE = "lineage";
private static final StringSet SOURCES =
Metrics.stringSet(LINEAGE_NAMESPACE, Type.SOURCE.toString());
private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, Type.SINK.toString());
private static final Lineage SOURCES = new Lineage(Type.SOURCE);
private static final Lineage SINKS = new Lineage(Type.SINK);
private static final Pattern RESERVED_CHARS = Pattern.compile("[:\\s.]");

private final StringSet metric;

/** {@link StringSet} representing sources and optionally side inputs. */
public static StringSet getSources() {
private Lineage(Type type) {
this.metric = Metrics.stringSet(LINEAGE_NAMESPACE, type.toString());
}

/** {@link Lineage} representing sources and optionally side inputs. */
public static Lineage getSources() {
return SOURCES;
}

/** {@link StringSet} representing sinks. */
public static StringSet getSinks() {
/** {@link Lineage} representing sinks. */
public static Lineage getSinks() {
return SINKS;
}

/**
* Wrap segment to valid segment name.
*
* <p>Specifically, If there are reserved chars (colon, whitespace, dot), escape with backtick. If
* the segment is already wrapped, return the original.
*/
private static String wrapSegment(String value) {
if (value.startsWith("`") && value.endsWith("`")) {
return value;
}
if (RESERVED_CHARS.matcher(value).find()) {
return String.format("`%s`", value);
}
return value;
}

/**
* Assemble fully qualified name (<a
* href="https://cloud.google.com/data-catalog/docs/fully-qualified-names">FQN</a>). Format:
*
* <ul>
* <li>{@code system:segment1.segment2}
* <li>{@code system:routine:segment1.segment2}
* <li>{@code system:`segment1.with.dots:clons`.segment2}
* </ul>
*
* <p>This helper method is for internal and testing usage only.
*/
@Internal
public static String getFqName(
String system, @Nullable String routine, Iterable<String> segments) {
StringBuilder builder = new StringBuilder(system);
if (!Strings.isNullOrEmpty(routine)) {
builder.append(":").append(routine);
}
int idx = 0;
for (String segment : segments) {
if (idx == 0) {
builder.append(":");
} else {
builder.append(".");
}
builder.append(wrapSegment(segment));
++idx;
}
return builder.toString();
}

/**
* Assemble the FQN of given system, and segments.
*
* <p>This helper method is for internal and testing usage only.
*/
@Internal
public static String getFqName(String system, Iterable<String> segments) {
return getFqName(system, null, segments);
}

/**
* Add a FQN (fully-qualified name) to Lineage. Segments will be processed via {@link #getFqName}.
*/
public void add(String system, @Nullable String routine, Iterable<String> segments) {
metric.add(getFqName(system, routine, segments));
}

/**
* Add a FQN (fully-qualified name) to Lineage. Segments will be processed via {@link #getFqName}.
*/
public void add(String system, Iterable<String> segments) {
add(system, null, segments);
}

/** Query {@link StringSet} metrics from {@link MetricResults}. */
public static Set<String> query(MetricResults results, Type type) {
MetricsFilter filter =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.metrics;

import static org.junit.Assert.assertEquals;

import java.util.Map;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Tests for {@link Lineage}. */
@RunWith(JUnit4.class)
public class LineageTest {
@Test
public void testGetFqName() {
Map<String, String> testCases =
ImmutableMap.<String, String>builder()
.put("apache-beam", "apache-beam")
.put("`apache-beam`", "`apache-beam`")
.put("apache.beam", "`apache.beam`")
.put("apache:beam", "`apache:beam`")
.put("apache beam", "`apache beam`")
.put("`apache beam`", "`apache beam`")
.put("apache\tbeam", "`apache\tbeam`")
.put("apache\nbeam", "`apache\nbeam`")
.build();
testCases.forEach(
(key, value) ->
assertEquals("apache:" + value, Lineage.getFqName("apache", ImmutableList.of(key))));
testCases.forEach(
(key, value) ->
assertEquals(
"apache:beam:" + value,
Lineage.getFqName("apache", "beam", ImmutableList.of(key))));
testCases.forEach(
(key, value) ->
assertEquals(
"apache:beam:" + value + "." + value,
Lineage.getFqName("apache", "beam", ImmutableList.of(key, key))));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.beam.sdk.io.fs.MatchResult.Status;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch;
Expand Down Expand Up @@ -214,6 +215,16 @@ protected String getScheme() {
return "gs";
}

@Override
protected void reportLineage(GcsResourceId resourceId, Lineage lineage) {
GcsPath path = resourceId.getGcsPath();
if (!path.getBucket().isEmpty()) {
lineage.add("gcs", ImmutableList.of(path.getBucket(), path.getObject()));
} else {
LOG.warn("Report Lineage on relative path {} is unsupported", path.getObject());
}
}

private List<MatchResult> matchGlobs(List<GcsPath> globs) {
// TODO: Executes in parallel, address https://issues.apache.org/jira/browse/BEAM-1503.
return FluentIterable.from(globs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.apache.beam.sdk.io.fs.CreateOptions;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.util.MoreFutures;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
Expand Down Expand Up @@ -624,6 +625,11 @@ protected S3ResourceId matchNewResource(String singleResourceSpec, boolean isDir
return S3ResourceId.fromUri(singleResourceSpec);
}

@Override
protected void reportLineage(S3ResourceId resourceId, Lineage lineage) {
lineage.add("s3", ImmutableList.of(resourceId.getBucket(), resourceId.getKey()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is the relative path not possible here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For GCS there is a GcsPath that also handles relative path. GcsResourceId is a wrapper of it so in theory there is possibility encounter relative path, that's why I added a warning in GcsFileSystem.reportLineage.

Current codepath should never encounter relative gcs path in reportLineage as the resourceId parsed in are all matched result that was assembled from GcsPath.fromObject(storageObject), where storageObject comes from List API call response, which then resolved to full path.

For s3 FileSystem it's not possible. S3ResourceId stores the absolute path directly (there is no equivalent of GcsPath here). There is essentially single entrance to new an S3ResourceId object which is here:

static S3ResourceId fromComponents(String scheme, String bucket, String key) {

and it explicitly add a "/" in key.

}

/**
* Invokes tasks in a thread pool, then unwraps the resulting {@link Future Futures}.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.beam.sdk.io.fs.CreateOptions;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.util.MoreFutures;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
Expand Down Expand Up @@ -655,6 +656,11 @@ protected S3ResourceId matchNewResource(String singleResourceSpec, boolean isDir
return S3ResourceId.fromUri(singleResourceSpec);
}

@Override
protected void reportLineage(S3ResourceId resourceId, Lineage lineage) {
lineage.add("s3", ImmutableList.of(resourceId.getBucket(), resourceId.getKey()));
}

/**
* Invokes tasks in a thread pool, then unwraps the resulting {@link Future Futures}.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import org.apache.beam.sdk.io.fs.CreateOptions;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.util.InstanceBuilder;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
Expand Down Expand Up @@ -449,4 +450,16 @@ protected AzfsResourceId matchNewResource(String singleResourceSpec, boolean isD
}
return AzfsResourceId.fromUri(singleResourceSpec);
}

@Override
protected void reportLineage(AzfsResourceId resourceId, Lineage lineage) {
if (!Strings.isNullOrEmpty(resourceId.getBlob())) {
lineage.add(
"abs",
ImmutableList.of(
resourceId.getAccount(), resourceId.getContainer(), resourceId.getBlob()));
} else {
lineage.add("abs", ImmutableList.of(resourceId.getAccount(), resourceId.getContainer()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import static org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment;
import static org.apache.beam.sdk.io.common.FileBasedIOITHelper.appendTimestampSuffix;
import static org.apache.beam.sdk.io.common.FileBasedIOITHelper.readFileBasedIOITPipelineOptions;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;

import java.time.Instant;
Expand All @@ -36,6 +37,7 @@
import org.apache.beam.sdk.io.common.FileBasedIOITHelper.DeleteFileFn;
import org.apache.beam.sdk.io.common.FileBasedIOTestPipelineOptions;
import org.apache.beam.sdk.io.common.HashingFn;
import org.apache.beam.sdk.metrics.Lineage;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testutils.NamedTestResult;
Expand Down Expand Up @@ -152,6 +154,9 @@ public void writeThenReadAll() {

PipelineResult result = pipeline.run();
PipelineResult.State pipelineState = result.waitUntilFinish();
assertEquals(
Lineage.query(result.metrics(), Lineage.Type.SOURCE),
Lineage.query(result.metrics(), Lineage.Type.SINK));

collectAndPublishMetrics(result);
// Fail the test if pipeline failed.
Expand Down
Loading
Loading