-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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 for BigtableIO #32068
Changes from 3 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,17 +17,17 @@ | |
*/ | ||
package org.apache.beam.sdk.metrics; | ||
|
||
import java.util.HashSet; | ||
import java.util.Set; | ||
|
||
/** | ||
* Standard collection of metrics used to record source and sinks information for lineage tracking. | ||
*/ | ||
public class Lineage { | ||
|
||
public static final String LINEAGE_NAMESPACE = "lineage"; | ||
public static final String SOURCE_METRIC_NAME = "sources"; | ||
public static final String SINK_METRIC_NAME = "sinks"; | ||
|
||
private static final StringSet SOURCES = Metrics.stringSet(LINEAGE_NAMESPACE, SOURCE_METRIC_NAME); | ||
private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, SINK_METRIC_NAME); | ||
private static final StringSet SOURCES = | ||
Metrics.stringSet(LINEAGE_NAMESPACE, Type.SOURCE.toString()); | ||
private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, Type.SINK.toString()); | ||
|
||
/** {@link StringSet} representing sources and optionally side inputs. */ | ||
public static StringSet getSources() { | ||
|
@@ -38,4 +38,35 @@ public static StringSet getSources() { | |
public static StringSet getSinks() { | ||
return SINKS; | ||
} | ||
|
||
/** Query {@link StringSet} metrics from {@link MetricResults}. */ | ||
public static Set<String> query(MetricResults results, Type type) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nice! |
||
MetricsFilter filter = | ||
MetricsFilter.builder() | ||
.addNameFilter(MetricNameFilter.named(LINEAGE_NAMESPACE, type.toString())) | ||
.build(); | ||
Set<String> result = new HashSet<>(); | ||
for (MetricResult<StringSetResult> metrics : results.queryMetrics(filter).getStringSets()) { | ||
result.addAll(metrics.getCommitted().getStringSet()); | ||
result.addAll(metrics.getAttempted().getStringSet()); | ||
} | ||
return result; | ||
} | ||
|
||
/** Lineage metrics resource types. */ | ||
public enum Type { | ||
SOURCE("source"), | ||
SINK("sink"); | ||
|
||
private final String name; | ||
|
||
Type(String name) { | ||
this.name = name; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return name; | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -218,6 +218,10 @@ task integrationTest(type: Test, dependsOn: processTestResources) { | |
|
||
useJUnit { | ||
excludeCategories "org.apache.beam.sdk.testing.UsesKms" | ||
filter { | ||
// https://github.com/apache/beam/issues/32071 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. interesting..... |
||
excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableReadIT.testE2EBigtableSegmentRead' | ||
} | ||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -71,6 +71,7 @@ | |
import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; | ||
import org.apache.beam.sdk.io.range.ByteKeyRange; | ||
import org.apache.beam.sdk.metrics.Distribution; | ||
import org.apache.beam.sdk.metrics.Lineage; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. One of the challenges is that BigtableIO has a complicated fallback logic to resolve projectId / instanceId. From transform setting / Bigtable setting / pipeline options, and there is BigtableConfig/BigtableWriteOptions/BigtableDataSettings. This was partly due to client migration that supported both old and new config class. For this reason this PR chose to emit metrics at the low level (BigtableServiceImpl) where the worker actually talking to the server, and where projectId/instanceId has been resolved definitely There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you. |
||
import org.apache.beam.sdk.metrics.Metrics; | ||
import org.apache.beam.sdk.values.KV; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; | ||
|
@@ -212,6 +213,11 @@ public void close() { | |
exhausted = true; | ||
} | ||
} | ||
|
||
@Override | ||
public void reportLineage() { | ||
Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); | ||
} | ||
} | ||
|
||
@VisibleForTesting | ||
|
@@ -225,6 +231,9 @@ static class BigtableSegmentReaderImpl implements Reader { | |
private final int refillSegmentWaterMark; | ||
private final long maxSegmentByteSize; | ||
private ServiceCallMetric serviceCallMetric; | ||
private final String projectId; | ||
private final String instanceId; | ||
private final String tableId; | ||
|
||
private static class UpstreamResults { | ||
private final List<Row> rows; | ||
|
@@ -308,11 +317,19 @@ static BigtableSegmentReaderImpl create( | |
// Asynchronously refill buffer when there is 10% of the elements are left | ||
this.refillSegmentWaterMark = | ||
Math.max(1, (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE)); | ||
this.projectId = projectId; | ||
this.instanceId = instanceId; | ||
this.tableId = tableId; | ||
} | ||
|
||
@Override | ||
public void close() {} | ||
|
||
@Override | ||
public void reportLineage() { | ||
Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); | ||
} | ||
|
||
@Override | ||
public boolean start() throws IOException { | ||
future = fetchNextSegment(); | ||
|
@@ -578,6 +595,11 @@ public void writeSingleRecord(KV<ByteString, Iterable<Mutation>> record) throws | |
} | ||
} | ||
|
||
@Override | ||
public void reportLineage() { | ||
Lineage.getSinks().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); | ||
} | ||
|
||
private ServiceCallMetric createServiceCallMetric() { | ||
// Populate metrics | ||
HashMap<String, String> baseLabels = new HashMap<>(); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,9 @@ | |
*/ | ||
package org.apache.beam.sdk.io.gcp.bigtable; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.hasItem; | ||
|
||
import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; | ||
import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; | ||
import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; | ||
|
@@ -28,7 +31,9 @@ | |
import java.util.Date; | ||
import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; | ||
import org.apache.beam.sdk.Pipeline; | ||
import org.apache.beam.sdk.PipelineResult; | ||
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; | ||
import org.apache.beam.sdk.metrics.Lineage; | ||
import org.apache.beam.sdk.options.PipelineOptionsFactory; | ||
import org.apache.beam.sdk.testing.PAssert; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
|
@@ -110,7 +115,8 @@ public void testE2EBigtableRead() { | |
p.apply(BigtableIO.read().withBigtableOptions(bigtableOptionsBuilder).withTableId(tableId)) | ||
.apply(Count.globally()); | ||
PAssert.thatSingleton(count).isEqualTo(numRows); | ||
p.run(); | ||
PipelineResult r = p.run(); | ||
checkLineageSourceMetric(r, tableId); | ||
} | ||
|
||
@Test | ||
|
@@ -138,6 +144,17 @@ public void testE2EBigtableSegmentRead() { | |
.withMaxBufferElementCount(10)) | ||
.apply(Count.globally()); | ||
PAssert.thatSingleton(count).isEqualTo(numRows); | ||
p.run(); | ||
PipelineResult r = p.run(); | ||
checkLineageSourceMetric(r, tableId); | ||
} | ||
|
||
private void checkLineageSourceMetric(PipelineResult r, String tableId) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. one cannot check Lineage metrics in unit test in BigtableIO as what BigQueryIO does is because the metrics are emitted in ServiceImpl, where unit tests used a fake Reader + ServiceImpl which does not emit Lineage (and it does not make sense add Lineage metrics there as well, because it is completely differerent code path from the real Reader / ServiceImpl) |
||
// TODO(https://github.com/apache/beam/issues/32071) test malformed, | ||
// when pipeline.run() os non-blocking, the metrics are not available by the time of query | ||
Abacn marked this conversation as resolved.
Show resolved
Hide resolved
|
||
if (options.getRunner().getName().contains("DirectRunner")) { | ||
assertThat( | ||
Lineage.query(r.metrics(), Lineage.Type.SOURCE), | ||
hasItem(String.format("bigtable:%s.%s.%s", project, options.getInstanceId(), tableId))); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is breaking change but Lineage is just introduced for one release and only used internally. I would suggest change the constant String to a enum so we can have a query method guarantees the metric name is among defined ones
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SG. Thanks.