From 0d9c0b2506a5d06208881e48d25272aa93a9af70 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 18 Nov 2022 16:12:18 -0500 Subject: [PATCH 01/52] feat: use the propagation inject framework for OpenTelemetry publishes --- samples/openTelemetryTracing.js | 15 +-- src/index.ts | 2 + src/opentelemetry-tracing.ts | 211 ++++++++++++++++++++++++++++++-- src/publisher/index.ts | 49 +++----- test/opentelemetry-tracing.ts | 103 +++++++++++++++- test/publisher/index.ts | 5 +- 6 files changed, 334 insertions(+), 51 deletions(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index dae84bdcf..6ad4d0259 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -49,7 +49,7 @@ function main( const {PubSub} = require('@google-cloud/pubsub'); // Imports the OpenTelemetry API - const opentelemetry = require('@opentelemetry/api'); + const openTelemetryApi = require('@opentelemetry/api'); // Imports the OpenTelemetry span handlers and exporter const { @@ -62,10 +62,11 @@ function main( const provider = new BasicTracerProvider(); const exporter = new ConsoleSpanExporter(); provider.addSpanProcessor(new SimpleSpanProcessor(exporter)); - // Enable the diagnostic logger for Opentelemetry - opentelemetry.diag.setLogger( - new opentelemetry.DiagConsoleLogger(), - opentelemetry.DiagLogLevel.INFO + + // Enable the diagnostic logger for OpenTelemetry + openTelemetryApi.diag.setLogger( + new openTelemetryApi.DiagConsoleLogger(), + openTelemetryApi.DiagLogLevel.INFO ); provider.register(); @@ -76,7 +77,7 @@ function main( enableOpenTelemetryTracing: true, }; - // Creates a client; cache this for further use + // Creates a client; cache this for further use. const pubSubClient = new PubSub(); async function publishMessage() { @@ -84,7 +85,7 @@ function main( const dataBuffer = Buffer.from(data); const messageId = await pubSubClient .topic(topicNameOrId, enableOpenTelemetryTracing) - .publish(dataBuffer); + .publishMessage({data: dataBuffer}); console.log(`Message ${messageId} published.`); } diff --git a/src/index.ts b/src/index.ts index 787b126ee..cb3f97a19 100644 --- a/src/index.ts +++ b/src/index.ts @@ -183,3 +183,5 @@ if (process.env.DEBUG_GRPC) { } import * as protos from '../protos/protos'; export {protos}; + +export * as openTelemetry from './opentelemetry-tracing'; diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index df3836f72..e121fc688 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -15,32 +15,162 @@ import { Tracer, - SpanAttributes, SpanContext, Span, context, trace, + propagation, SpanKind, + TextMapGetter, + TextMapSetter, + ROOT_CONTEXT, + Context, } from '@opentelemetry/api'; +import {Attributes} from './publisher/pubsub-message'; +import {PublishOptions} from './publisher/index'; +// We need this to get the library version. // eslint-disable-next-line @typescript-eslint/no-var-requires -const PKG = require('../../package.json'); +const packageJson = require('../../package.json'); /** - * @internal * Instantiates a Opentelemetry tracer for the library + * + * @private */ const libraryTracer: Tracer = trace.getTracer( '@google-cloud/pubsub', - PKG.version + packageJson.version ); +/** + * Determination of the level of OTel support we're providing. + * + * @private + */ +export enum OpenTelemetryLevel { + /** + * None: OTel support is not enabled because we found no trace provider. + */ + None = 0, + + /** + * Legacy: We found a trace provider, but the user also specified the old + * manual enable flag; this will trigger the legacy attribute being included. + * The modern propagation attribute will _also_ be included. + */ + Legacy = 1, + + /** + * Modern: We will only inject/extract the modern propagation attribute. + */ + Modern = 2, +} + +/** + * Tries to divine what sort of OpenTelemetry we're supporting. See the enum + * for the meaning of the values, and other notes. + * + * Legacy OTel is no longer officially supported, but we don't want to + * break anyone at a non-major. + * + * @private + */ +export function isEnabled( + publishSettings?: PublishOptions +): OpenTelemetryLevel { + // If there's no trace provider attached, do nothing in any case. + const traceProvider = trace.getTracerProvider(); + if (!traceProvider) { + return OpenTelemetryLevel.None; + } + + if (publishSettings?.enableOpenTelemetryTracing) { + return OpenTelemetryLevel.Legacy; + } + + return OpenTelemetryLevel.Modern; +} + +/** + * Our Carrier object for propagation is anything with an 'attributes' + * object, which is one of several possible Message classes. (They're + * different for publish and subscribe.) + * + * @private + */ +interface MessageWithAttributes { + attributes?: Attributes | null | undefined; +} + +/** + * Implements the TextMap getter and setter interfaces for Pub/Sub messages. + * + * @private + */ +export class PubsubMessageGetSet + implements + TextMapGetter, + TextMapSetter +{ + static keyPrefix = 'googclient_'; + + keys(carrier: MessageWithAttributes): string[] { + return Object.getOwnPropertyNames(carrier.attributes) + .filter(n => n.startsWith(PubsubMessageGetSet.keyPrefix)) + .map(n => n.substring(PubsubMessageGetSet.keyPrefix.length)); + } + + private attributeName(key: string): string { + return `${PubsubMessageGetSet.keyPrefix}${key}`; + } + + get( + carrier: MessageWithAttributes, + key: string + ): string | string[] | undefined { + return carrier?.attributes?.[this.attributeName(key)]; + } + + set(carrier: MessageWithAttributes, key: string, value: string): void { + if (!carrier.attributes) { + carrier.attributes = {}; + } + carrier.attributes[this.attributeName(key)] = value; + } +} + +/** + * The getter to use when calling extract() on a Pub/Sub message. + * + * @private + */ +export const pubsubGetter = new PubsubMessageGetSet(); + +/** + * The setter to use when calling inject() on a Pub/Sub message. + * + * @private + */ +export const pubsubSetter = pubsubGetter; + +/** + * Description of the data structure passed for span attributes. + * + * @private + */ +export interface SpanAttributes { + [x: string]: string | number; +} + /** * Creates a new span with the given properties * * @param {string} spanName the name for the span * @param {Attributes?} attributes an object containing the attributes to be set for the span * @param {SpanContext?} parent the context of the parent span to link to the span + * + * @private */ export function createSpan( spanName: string, @@ -51,11 +181,78 @@ export function createSpan( return libraryTracer.startSpan( spanName, { - // set the kind of the span kind, - // set the attributes of the span - attributes: attributes, + attributes, }, parent ? trace.setSpanContext(context.active(), parent) : undefined ); } + +/** + * The modern propagation attribute name. + * + * Technically this is determined by the OpenTelemetry library, but + * in practice, it follows the W3C spec, so this should be the right + * one. The only thing we're using it for, anyway, is emptying user + * supplied attributes. + * + * @private + */ +export const modernAttributeName = 'googclient_traceparent'; + +/** + * The old legacy attribute name. + * + * @private + */ +export const legacyAttributeName = 'googclient_OpenTelemetrySpanContext'; + +/** + * Injects the trace context into a Pub/Sub message (or other object with + * an 'attributes' object) for propagation. + * + * @private + */ +export function injectSpan( + span: Span, + message: MessageWithAttributes, + enabled: OpenTelemetryLevel +): void { + if (!message.attributes) { + message.attributes = {}; + } + + if (message.attributes[modernAttributeName]) { + console.warn( + `${modernAttributeName} key set as message attribute, but will be overridden.` + ); + + delete message.attributes[modernAttributeName]; + } + + // If we're in legacy mode, add that header as well. + if (enabled === OpenTelemetryLevel.Legacy) { + if (message.attributes[legacyAttributeName]) { + console.warn( + `${legacyAttributeName} key set as message attribute, but will be overridden.` + ); + } + message.attributes[legacyAttributeName] = JSON.stringify( + span.spanContext() + ); + } + + // Always do propagation injection with the trace context. + const context = trace.setSpanContext(ROOT_CONTEXT, span.spanContext()); + propagation.inject(context, message, pubsubSetter); +} + +/** + * Extracts the trace context from a Pub/Sub message (or other object with + * an 'attributes' object) from a propagation. + * + * @private + */ +export function extractSpan(carrier: MessageWithAttributes): Context { + return propagation.extract(ROOT_CONTEXT, carrier, pubsubGetter); +} diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 1a545784e..d7d3436e5 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -25,7 +25,7 @@ import {Queue, OrderedQueue} from './message-queues'; import {Topic} from '../topic'; import {RequestCallback, EmptyCallback} from '../pubsub'; import {defaultOptions} from '../default-options'; -import {createSpan} from '../opentelemetry-tracing'; +import * as otel from '../opentelemetry-tracing'; import {FlowControl, FlowControlOptions} from './flow-control'; import {promisifySome} from '../util'; @@ -215,23 +215,19 @@ export class Publisher { if (!message.orderingKey) { this.queue.add(message, callback!); - if (span) { - span.end(); - } - return; - } + } else { + const key = message.orderingKey; - const key = message.orderingKey; + if (!this.orderedQueues.has(key)) { + const queue = new OrderedQueue(this, key); + this.orderedQueues.set(key, queue); + queue.once('drain', () => this.orderedQueues.delete(key)); + } - if (!this.orderedQueues.has(key)) { - const queue = new OrderedQueue(this, key); - this.orderedQueues.set(key, queue); - queue.once('drain', () => this.orderedQueues.delete(key)); + const queue = this.orderedQueues.get(key)!; + queue.add(message, callback!); } - const queue = this.orderedQueues.get(key)!; - queue.add(message, callback!); - if (span) { span.end(); } @@ -341,7 +337,9 @@ export class Publisher { * @param {PubsubMessage} message The message to create a span for */ constructSpan(message: PubsubMessage): Span | undefined { - if (!this.settings.enableOpenTelemetryTracing) { + const enabled = otel.isEnabled(this.settings); + + if (!enabled) { return undefined; } @@ -358,30 +356,17 @@ export class Publisher { [SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES]: message.data?.length, 'messaging.pubsub.ordering_key': message.orderingKey, - } as Attributes; + } as otel.SpanAttributes; - const span: Span = createSpan( + const span: Span = otel.createSpan( `${this.topic.name} send`, SpanKind.PRODUCER, spanAttributes ); - // If the span's context is valid we should pass the span context special attribute + // If the span's context is valid we should inject the propagation trace context. if (isSpanContextValid(span.spanContext())) { - if ( - message.attributes && - message.attributes['googclient_OpenTelemetrySpanContext'] - ) { - console.warn( - 'googclient_OpenTelemetrySpanContext key set as message attribute, but will be overridden.' - ); - } - if (!message.attributes) { - message.attributes = {}; - } - - message.attributes['googclient_OpenTelemetrySpanContext'] = - JSON.stringify(span.spanContext()); + otel.injectSpan(span, message, enabled); } return span; diff --git a/test/opentelemetry-tracing.ts b/test/opentelemetry-tracing.ts index dc6a1423e..2653c8e9e 100644 --- a/test/opentelemetry-tracing.ts +++ b/test/opentelemetry-tracing.ts @@ -19,9 +19,10 @@ import {describe, it, beforeEach} from 'mocha'; import * as api from '@opentelemetry/api'; import * as trace from '@opentelemetry/tracing'; -import {createSpan} from '../src/opentelemetry-tracing'; +import * as otel from '../src/opentelemetry-tracing'; import {exporter} from './tracing'; import {SpanKind} from '@opentelemetry/api'; +import sinon = require('sinon'); describe('OpenTelemetryTracer', () => { let span: trace.Span; @@ -31,7 +32,7 @@ describe('OpenTelemetryTracer', () => { spanId: '6e0c63257de34c92', traceFlags: api.TraceFlags.SAMPLED, }; - const spanAttributes: api.SpanAttributes = { + const spanAttributes: otel.SpanAttributes = { foo: 'bar', }; @@ -40,7 +41,7 @@ describe('OpenTelemetryTracer', () => { }); it('creates a span', () => { - span = createSpan( + span = otel.createSpan( spanName, SpanKind.PRODUCER, spanAttributes, @@ -57,4 +58,100 @@ describe('OpenTelemetryTracer', () => { assert.strictEqual(exportedSpan.parentSpanId, spanContext.spanId); assert.strictEqual(exportedSpan.kind, SpanKind.PRODUCER); }); + + it('injects a trace context', () => { + span = otel.createSpan( + spanName, + SpanKind.PRODUCER, + spanAttributes, + spanContext + ) as trace.Span; + + const message = { + attributes: {}, + }; + + otel.injectSpan(span, message, otel.OpenTelemetryLevel.Modern); + + assert.strictEqual( + Object.getOwnPropertyNames(message.attributes).includes( + otel.modernAttributeName + ), + true + ); + }); + + it('injects a trace context and legacy baggage', () => { + span = otel.createSpan( + spanName, + SpanKind.PRODUCER, + spanAttributes, + spanContext + ) as trace.Span; + + const message = { + attributes: {}, + }; + + otel.injectSpan(span, message, otel.OpenTelemetryLevel.Legacy); + assert.strictEqual( + Object.getOwnPropertyNames(message.attributes).includes( + otel.modernAttributeName + ), + true + ); + assert.strictEqual( + Object.getOwnPropertyNames(message.attributes).includes( + otel.legacyAttributeName + ), + true + ); + }); + + it('should issue a warning if OpenTelemetry span context key is set', () => { + span = otel.createSpan( + spanName, + SpanKind.PRODUCER, + spanAttributes, + spanContext + ) as trace.Span; + + const warnSpy = sinon.spy(console, 'warn'); + try { + const message = { + attributes: { + [otel.legacyAttributeName]: 'foobar', + [otel.modernAttributeName]: 'bazbar', + }, + }; + otel.injectSpan(span, message, otel.OpenTelemetryLevel.Legacy); + assert.strictEqual(warnSpy.callCount, 2); + } finally { + warnSpy.restore(); + } + }); + + it('extracts a trace context', () => { + span = otel.createSpan( + spanName, + SpanKind.PRODUCER, + spanAttributes, + spanContext + ) as trace.Span; + + const message = { + attributes: { + [otel.modernAttributeName]: + '00-d4cda95b652f4a1592b449d5929fda1b-553964cd9101a314-01', + }, + }; + + const libraryTracer = api.trace.getTracer('@google-cloud/pubsub', '1.0.0'); + const parentCtx = otel.extractSpan(message); + const childSpan = libraryTracer.startSpan('child', undefined, parentCtx); + assert.strictEqual( + childSpan.spanContext().traceId, + 'd4cda95b652f4a1592b449d5929fda1b' + ); + }); }); diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 151d1b96e..42d89602d 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -28,6 +28,7 @@ import {PublishError} from '../../src/publisher/publish-error'; import * as util from '../../src/util'; import {defaultOptions} from '../../src/default-options'; +import * as otel from '../../src/opentelemetry-tracing'; import {exporter} from '../tracing'; import {SpanKind} from '@opentelemetry/api'; import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; @@ -196,7 +197,7 @@ describe('Publisher', () => { // Setup trace exporting tracingPublisher = new Publisher(topic, enableTracing); - tracingPublisher.publish(buffer); + tracingPublisher.publishMessage({data: buffer}); const spans = exporter.getFinishedSpans(); assert.notStrictEqual(spans.length, 0, 'has span'); const createdSpan = spans.concat().pop()!; @@ -383,7 +384,7 @@ describe('Publisher', () => { it('should issue a warning if OpenTelemetry span context key is set', () => { const warnSpy = sinon.spy(console, 'warn'); const attributes = { - googclient_OpenTelemetrySpanContext: 'foobar', + [otel.legacyAttributeName]: 'foobar', }; const fakeMessageWithOTKey = {data, attributes}; const publisherTracing = new Publisher(topic, { From 4f3daa995164aae59a021d797aec0f762da551f9 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 18 Nov 2022 17:50:39 -0500 Subject: [PATCH 02/52] feat: add propagation extraction to subscriber for OpenTelemetry --- src/opentelemetry-tracing.ts | 73 ++++++++++++++++++++++++++++++++--- src/publisher/index.ts | 2 + src/subscriber.ts | 42 ++++++++++---------- test/opentelemetry-tracing.ts | 41 +++++++++++++++----- test/subscriber.ts | 10 ++--- 5 files changed, 126 insertions(+), 42 deletions(-) diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index e121fc688..3217ca22c 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -99,7 +99,7 @@ export function isEnabled( * * @private */ -interface MessageWithAttributes { +export interface MessageWithAttributes { attributes?: Attributes | null | undefined; } @@ -168,7 +168,7 @@ export interface SpanAttributes { * * @param {string} spanName the name for the span * @param {Attributes?} attributes an object containing the attributes to be set for the span - * @param {SpanContext?} parent the context of the parent span to link to the span + * @param {Context?} parent the context of the parent span to link to the span * * @private */ @@ -176,7 +176,7 @@ export function createSpan( spanName: string, kind: SpanKind, attributes?: SpanAttributes, - parent?: SpanContext + parent?: Context ): Span { return libraryTracer.startSpan( spanName, @@ -184,10 +184,21 @@ export function createSpan( kind, attributes, }, - parent ? trace.setSpanContext(context.active(), parent) : undefined + parent ); } +/** + * Converts a SpanContext to a full Context, as needed. + * + * @private + */ +export function spanContextToContext( + parent?: SpanContext +): Context | undefined { + return parent ? trace.setSpanContext(context.active(), parent) : undefined; +} + /** * The modern propagation attribute name. * @@ -247,12 +258,62 @@ export function injectSpan( propagation.inject(context, message, pubsubSetter); } +/** + * Returns true if this message potentially contains a span context attribute. + * + * @private + */ +export function containsSpanContext(message: MessageWithAttributes): boolean { + if (!message.attributes) { + return false; + } + + const keys = Object.getOwnPropertyNames(message.attributes); + return !!keys.find( + n => n === legacyAttributeName || n === modernAttributeName + ); +} + /** * Extracts the trace context from a Pub/Sub message (or other object with * an 'attributes' object) from a propagation. * * @private */ -export function extractSpan(carrier: MessageWithAttributes): Context { - return propagation.extract(ROOT_CONTEXT, carrier, pubsubGetter); +export function extractSpan( + message: MessageWithAttributes, + spanName: string, + spanAttributes: SpanAttributes, + enabled: OpenTelemetryLevel +): Span | undefined { + if (!message.attributes) { + return undefined; + } + const keys = Object.getOwnPropertyNames(message.attributes); + + let context: Context | undefined; + + if (enabled === OpenTelemetryLevel.Legacy) { + // Only prefer the legacy attributes to no trace context attribute. + if ( + keys.includes(legacyAttributeName) && + !keys.includes(modernAttributeName) + ) { + const legacyValue = message.attributes[legacyAttributeName]; + const parentSpanContext: SpanContext | undefined = legacyValue + ? JSON.parse(legacyValue) + : undefined; + if (parentSpanContext) { + context = spanContextToContext(parentSpanContext); + } + } + } else { + if (keys.includes(modernAttributeName)) { + context = propagation.extract(ROOT_CONTEXT, message, pubsubGetter); + } + } + + return context + ? createSpan(spanName, SpanKind.CONSUMER, spanAttributes, context) + : undefined; } diff --git a/src/publisher/index.ts b/src/publisher/index.ts index d7d3436e5..76c992597 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -40,6 +40,8 @@ export interface PublishOptions { flowControlOptions?: FlowControlOptions; gaxOpts?: CallOptions; messageOrdering?: boolean; + + /** @deprecated Unset and use context propagation. */ enableOpenTelemetryTracing?: boolean; } diff --git a/src/subscriber.ts b/src/subscriber.ts index 2011aad63..c7137ad78 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -29,7 +29,7 @@ import {MessageStream, MessageStreamOptions} from './message-stream'; import {Subscription} from './subscription'; import {defaultOptions} from './default-options'; import {SubscriberClient} from './v1'; -import {createSpan} from './opentelemetry-tracing'; +import * as otel from './opentelemetry-tracing'; import {Duration} from './temporal'; export type PullResponse = google.pubsub.v1.IStreamingPullResponse; @@ -341,6 +341,8 @@ export interface SubscriberOptions { flowControl?: FlowControlOptions; useLegacyFlowControl?: boolean; streamingOptions?: MessageStreamOptions; + + /** @deprecated Unset and use context propagation. */ enableOpenTelemetryTracing?: boolean; } @@ -364,7 +366,7 @@ export class Subscriber extends EventEmitter { private _acks!: AckQueue; private _histogram: Histogram; private _inventory!: LeaseManager; - private _useOpentelemetry: boolean; + private _useLegacyOpenTelemetry: boolean; private _latencies: Histogram; private _modAcks!: ModAckQueue; private _name!: string; @@ -382,7 +384,7 @@ export class Subscriber extends EventEmitter { this.maxBytes = defaultOptions.subscription.maxOutstandingBytes; this.useLegacyFlowControl = false; this.isOpen = false; - this._useOpentelemetry = false; + this._useLegacyOpenTelemetry = false; this._histogram = new Histogram({min: 10, max: 600}); this._latencies = new Histogram(); this._subscription = subscription; @@ -699,7 +701,7 @@ export class Subscriber extends EventEmitter { setOptions(options: SubscriberOptions): void { this._options = options; - this._useOpentelemetry = options.enableOpenTelemetryTracing || false; + this._useLegacyOpenTelemetry = options.enableOpenTelemetryTracing || false; // The user-set ackDeadline value basically pegs the extension time. // We'll emulate it by overwriting min/max. @@ -744,18 +746,21 @@ export class Subscriber extends EventEmitter { */ private _constructSpan(message: Message): Span | undefined { // Handle cases where OpenTelemetry is disabled or no span context was sent through message - if ( - !this._useOpentelemetry || - !message.attributes || - !message.attributes['googclient_OpenTelemetrySpanContext'] - ) { + if (!this._useLegacyOpenTelemetry || !message.attributes) { + return undefined; + } + + const enabled = otel.isEnabled({ + enableOpenTelemetryTracing: this._useLegacyOpenTelemetry, + }); + if (!enabled) { + return undefined; + } + + if (!otel.containsSpanContext(message)) { return undefined; } - const spanValue = message.attributes['googclient_OpenTelemetrySpanContext']; - const parentSpanContext: SpanContext | undefined = spanValue - ? JSON.parse(spanValue) - : undefined; const spanAttributes = { // Original span attributes ackId: message.ackId, @@ -780,15 +785,8 @@ export class Subscriber extends EventEmitter { // Subscriber spans should always have a publisher span as a parent. // Return undefined if no parent is provided const spanName = `${this.name} process`; - const span = parentSpanContext - ? createSpan( - spanName.trim(), - SpanKind.CONSUMER, - spanAttributes, - parentSpanContext - ) - : undefined; - return span; + + return otel.extractSpan(message, spanName, spanAttributes, enabled); } /** diff --git a/test/opentelemetry-tracing.ts b/test/opentelemetry-tracing.ts index 2653c8e9e..8784ad11b 100644 --- a/test/opentelemetry-tracing.ts +++ b/test/opentelemetry-tracing.ts @@ -35,6 +35,7 @@ describe('OpenTelemetryTracer', () => { const spanAttributes: otel.SpanAttributes = { foo: 'bar', }; + const context = otel.spanContextToContext(spanContext); beforeEach(() => { exporter.reset(); @@ -45,7 +46,7 @@ describe('OpenTelemetryTracer', () => { spanName, SpanKind.PRODUCER, spanAttributes, - spanContext + context ) as trace.Span; span.end(); @@ -64,7 +65,7 @@ describe('OpenTelemetryTracer', () => { spanName, SpanKind.PRODUCER, spanAttributes, - spanContext + context ) as trace.Span; const message = { @@ -86,7 +87,7 @@ describe('OpenTelemetryTracer', () => { spanName, SpanKind.PRODUCER, spanAttributes, - spanContext + context ) as trace.Span; const message = { @@ -113,7 +114,7 @@ describe('OpenTelemetryTracer', () => { spanName, SpanKind.PRODUCER, spanAttributes, - spanContext + context ) as trace.Span; const warnSpy = sinon.spy(console, 'warn'); @@ -131,12 +132,31 @@ describe('OpenTelemetryTracer', () => { } }); + it('should be able to determine if attributes are present', () => { + let message: otel.MessageWithAttributes = { + attributes: { + [otel.legacyAttributeName]: 'foobar', + }, + }; + assert.strictEqual(otel.containsSpanContext(message), true); + + message = { + attributes: { + [otel.modernAttributeName]: 'foobar', + }, + }; + assert.strictEqual(otel.containsSpanContext(message), true); + + message = {}; + assert.strictEqual(otel.containsSpanContext(message), false); + }); + it('extracts a trace context', () => { span = otel.createSpan( spanName, SpanKind.PRODUCER, spanAttributes, - spanContext + context ) as trace.Span; const message = { @@ -146,11 +166,14 @@ describe('OpenTelemetryTracer', () => { }, }; - const libraryTracer = api.trace.getTracer('@google-cloud/pubsub', '1.0.0'); - const parentCtx = otel.extractSpan(message); - const childSpan = libraryTracer.startSpan('child', undefined, parentCtx); + const childSpan = otel.extractSpan( + message, + 'child', + {}, + otel.OpenTelemetryLevel.Modern + ); assert.strictEqual( - childSpan.spanContext().traceId, + childSpan!.spanContext().traceId, 'd4cda95b652f4a1592b449d5929fda1b' ); }); diff --git a/test/subscriber.ts b/test/subscriber.ts index 2e31c2d05..1bed1e346 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -733,31 +733,31 @@ describe('Subscriber', () => { it('should not instantiate a tracer when tracing is disabled', () => { subscriber = new Subscriber(subscription, {}); - assert.strictEqual(subscriber['_useOpentelemetry'], false); + assert.strictEqual(subscriber['_useLegacyOpenTelemetry'], false); }); it('should instantiate a tracer when tracing is enabled through constructor', () => { subscriber = new Subscriber(subscription, enableTracing); - assert.ok(subscriber['_useOpentelemetry']); + assert.ok(subscriber['_useLegacyOpenTelemetry']); }); it('should instantiate a tracer when tracing is enabled through setOptions', () => { subscriber = new Subscriber(subscription, {}); subscriber.setOptions(enableTracing); - assert.ok(subscriber['_useOpentelemetry']); + assert.ok(subscriber['_useLegacyOpenTelemetry']); }); it('should disable tracing when tracing is disabled through setOptions', () => { subscriber = new Subscriber(subscription, enableTracing); subscriber.setOptions(disableTracing); - assert.strictEqual(subscriber['_useOpentelemetry'], false); + assert.strictEqual(subscriber['_useLegacyOpenTelemetry'], false); }); it('exports a span once it is created', () => { subscription = new FakeSubscription() as {} as Subscription; subscriber = new Subscriber(subscription, enableTracing); message = new Message(subscriber, RECEIVED_MESSAGE); - assert.strictEqual(subscriber['_useOpentelemetry'], true); + assert.strictEqual(subscriber['_useLegacyOpenTelemetry'], true); subscriber.open(); // Construct mock of received message with span context From 80d8b6577f61b54e7d9d2c0d62215a844434accb Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Tue, 22 Nov 2022 17:41:23 -0500 Subject: [PATCH 03/52] feat: work in progress on adding sub-spans --- src/opentelemetry-tracing.ts | 202 +++++++++++++++++++++++++------- src/publisher/index.ts | 36 ++---- src/publisher/pubsub-message.ts | 5 +- src/subscriber.ts | 44 +------ test/opentelemetry-tracing.ts | 80 ++++--------- test/subscriber.ts | 35 ++---- 6 files changed, 209 insertions(+), 193 deletions(-) diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index 3217ca22c..bb2ad031d 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -26,8 +26,10 @@ import { ROOT_CONTEXT, Context, } from '@opentelemetry/api'; -import {Attributes} from './publisher/pubsub-message'; +import {Attributes, PubsubMessage} from './publisher/pubsub-message'; import {PublishOptions} from './publisher/index'; +import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; +import {Duration} from './temporal'; // We need this to get the library version. // eslint-disable-next-line @typescript-eslint/no-var-requires @@ -97,10 +99,14 @@ export function isEnabled( * object, which is one of several possible Message classes. (They're * different for publish and subscribe.) * + * Also we add a telemetrySpan optional member for passing around the + * actual Span object within the client library. + * * @private */ export interface MessageWithAttributes { attributes?: Attributes | null | undefined; + telemetrySpan?: Span; } /** @@ -163,31 +169,6 @@ export interface SpanAttributes { [x: string]: string | number; } -/** - * Creates a new span with the given properties - * - * @param {string} spanName the name for the span - * @param {Attributes?} attributes an object containing the attributes to be set for the span - * @param {Context?} parent the context of the parent span to link to the span - * - * @private - */ -export function createSpan( - spanName: string, - kind: SpanKind, - attributes?: SpanAttributes, - parent?: Context -): Span { - return libraryTracer.startSpan( - spanName, - { - kind, - attributes, - }, - parent - ); -} - /** * Converts a SpanContext to a full Context, as needed. * @@ -218,6 +199,132 @@ export const modernAttributeName = 'googclient_traceparent'; */ export const legacyAttributeName = 'googclient_OpenTelemetrySpanContext'; +export class SpanMaker { + static createPublisherSpan(message: PubsubMessage, topicName: string): Span { + const spanAttributes = { + // Add Opentelemetry semantic convention attributes to the span, based on: + // https://github.com/open-telemetry/opentelemetry-specification/blob/v1.1.0/specification/trace/semantic_conventions/messaging.md + [SemanticAttributes.MESSAGING_TEMP_DESTINATION]: false, + [SemanticAttributes.MESSAGING_SYSTEM]: 'pubsub', + [SemanticAttributes.MESSAGING_OPERATION]: 'send', + [SemanticAttributes.MESSAGING_DESTINATION]: topicName, + [SemanticAttributes.MESSAGING_DESTINATION_KIND]: 'topic', + [SemanticAttributes.MESSAGING_PROTOCOL]: 'pubsub', + [SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES]: + message.data?.length, + 'messaging.pubsub.ordering_key': message.orderingKey, + } as SpanAttributes; + + const span: Span = libraryTracer.startSpan(`${topicName} send`, { + kind: SpanKind.PRODUCER, + attributes: spanAttributes, + }); + + return span; + } + + static createReceiveSpan( + message: MessageWithAttributes, + subName: string, + parent: Context | undefined + ): Span { + const name = `${subName} receive`; + + // Mostly we want to keep the context IDs; the attributes and such + // are only something we do on the publish side. + if (context) { + return libraryTracer.startSpan( + name, + { + kind: SpanKind.CONSUMER, + attributes: {}, + }, + parent + ); + } else { + return libraryTracer.startSpan(name, { + kind: SpanKind.CONSUMER, + }); + } + } + + static createChildSpan( + message: MessageWithAttributes, + name: string + ): Span | undefined { + const parent = message.telemetrySpan; + if (parent) { + return libraryTracer.startSpan( + name, + { + kind: SpanKind.INTERNAL, + attributes: {}, + }, + spanContextToContext(parent.spanContext()) + ); + } else { + return undefined; + } + } + + static createPublishFlowSpan(message: PubsubMessage): Span | undefined { + return SpanMaker.createChildSpan(message, 'publisher flow control'); + } + + static createPublishBatchSpan(message: PubsubMessage): Span | undefined { + return SpanMaker.createChildSpan(message, 'publish scheduler'); + } + + static createPublishRpcSpan(message: PubsubMessage): Span | undefined { + return SpanMaker.createChildSpan(message, 'send Publish'); + } + + static createReceiveFlowSpan( + message: MessageWithAttributes + ): Span | undefined { + return SpanMaker.createChildSpan(message, 'subscriber flow control'); + } + + static createSchedulerSpan(message: MessageWithAttributes): Span | undefined { + return SpanMaker.createChildSpan(message, 'subscribe scheduler'); + } + + static createProcessSpan( + message: MessageWithAttributes, + subName: string + ): Span | undefined { + return SpanMaker.createChildSpan(message, `${subName} process`); + } + + static setProcessResult(span: Span, isAck: boolean) { + span.setAttribute('messaging.pubsub.result', isAck ? 'ack' : 'nack'); + } + + static createLeaseSpan( + message: MessageWithAttributes, + deadline: Duration, + isInitial: boolean + ): Span | undefined { + const span = SpanMaker.createChildSpan(message, 'send ModifyAckDeadline'); + span?.setAttribute( + 'messaging.pubsub.modack_deadline_seconds', + deadline.totalOf('second') + ); + span?.setAttribute('messaging.pubsub.is_receipt_modack', isInitial); + return span; + } + + static createResponseSpan( + message: MessageWithAttributes, + isAck: boolean + ): Span | undefined { + const name = isAck + ? 'send Acknowledgement' + : 'send Negative Acknowledgement'; + return SpanMaker.createChildSpan(message, name); + } +} + /** * Injects the trace context into a Pub/Sub message (or other object with * an 'attributes' object) for propagation. @@ -256,14 +363,22 @@ export function injectSpan( // Always do propagation injection with the trace context. const context = trace.setSpanContext(ROOT_CONTEXT, span.spanContext()); propagation.inject(context, message, pubsubSetter); + + // Also put the direct reference to the Span object for while we're + // passing it around in the client library. + message.telemetrySpan = span; } /** - * Returns true if this message potentially contains a span context attribute. + * Returns true if this message potentially contains a span context. * * @private */ export function containsSpanContext(message: MessageWithAttributes): boolean { + if (message.telemetrySpan) { + return true; + } + if (!message.attributes) { return false; } @@ -276,20 +391,21 @@ export function containsSpanContext(message: MessageWithAttributes): boolean { /** * Extracts the trace context from a Pub/Sub message (or other object with - * an 'attributes' object) from a propagation. + * an 'attributes' object) from a propagation, for receive processing. If no + * context was present, create a new parent span. * * @private */ export function extractSpan( message: MessageWithAttributes, - spanName: string, - spanAttributes: SpanAttributes, + subName: string, enabled: OpenTelemetryLevel ): Span | undefined { - if (!message.attributes) { - return undefined; + if (message.telemetrySpan) { + return message.telemetrySpan; } - const keys = Object.getOwnPropertyNames(message.attributes); + + const keys = Object.getOwnPropertyNames(message.attributes ?? {}); let context: Context | undefined; @@ -299,12 +415,14 @@ export function extractSpan( keys.includes(legacyAttributeName) && !keys.includes(modernAttributeName) ) { - const legacyValue = message.attributes[legacyAttributeName]; - const parentSpanContext: SpanContext | undefined = legacyValue - ? JSON.parse(legacyValue) - : undefined; - if (parentSpanContext) { - context = spanContextToContext(parentSpanContext); + const legacyValue = message.attributes?.[legacyAttributeName]; + if (legacyValue) { + const parentSpanContext: SpanContext | undefined = legacyValue + ? JSON.parse(legacyValue) + : undefined; + if (parentSpanContext) { + context = spanContextToContext(parentSpanContext); + } } } } else { @@ -313,7 +431,7 @@ export function extractSpan( } } - return context - ? createSpan(spanName, SpanKind.CONSUMER, spanAttributes, context) - : undefined; + const span = SpanMaker.createReceiveSpan(message, subName, context); + message.telemetrySpan = span; + return span; } diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 76c992597..66884a60d 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -17,8 +17,7 @@ import {promisify} from '@google-cloud/promisify'; import * as extend from 'extend'; import {CallOptions} from 'google-gax'; -import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; -import {isSpanContextValid, Span, SpanKind} from '@opentelemetry/api'; +import {isSpanContextValid, Span} from '@opentelemetry/api'; import {BatchPublishOptions} from './message-batch'; import {Queue, OrderedQueue} from './message-queues'; @@ -213,7 +212,7 @@ export class Publisher { } } - const span: Span | undefined = this.constructSpan(message); + const span: Span | undefined = this.getParentSpan(message); if (!message.orderingKey) { this.queue.add(message, callback!); @@ -332,39 +331,24 @@ export class Publisher { } /** - * Constructs an OpenTelemetry span + * Finds or constructs an OpenTelemetry publish/parent span for a message, + * if OTel is enabled. * * @private * * @param {PubsubMessage} message The message to create a span for */ - constructSpan(message: PubsubMessage): Span | undefined { + getParentSpan(message: PubsubMessage): Span | undefined { const enabled = otel.isEnabled(this.settings); - if (!enabled) { return undefined; } - const spanAttributes = { - // Add Opentelemetry semantic convention attributes to the span, based on: - // https://github.com/open-telemetry/opentelemetry-specification/blob/v1.1.0/specification/trace/semantic_conventions/messaging.md - [SemanticAttributes.MESSAGING_TEMP_DESTINATION]: false, - [SemanticAttributes.MESSAGING_SYSTEM]: 'pubsub', - [SemanticAttributes.MESSAGING_OPERATION]: 'send', - [SemanticAttributes.MESSAGING_DESTINATION]: this.topic.name, - [SemanticAttributes.MESSAGING_DESTINATION_KIND]: 'topic', - [SemanticAttributes.MESSAGING_MESSAGE_ID]: message.messageId, - [SemanticAttributes.MESSAGING_PROTOCOL]: 'pubsub', - [SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES]: - message.data?.length, - 'messaging.pubsub.ordering_key': message.orderingKey, - } as otel.SpanAttributes; - - const span: Span = otel.createSpan( - `${this.topic.name} send`, - SpanKind.PRODUCER, - spanAttributes - ); + if (message.telemetrySpan) { + return message.telemetrySpan; + } + + const span = otel.SpanMaker.createPublisherSpan(message, this.topic.name); // If the span's context is valid we should inject the propagation trace context. if (isSpanContextValid(span.spanContext())) { diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index a1e1283d5..269967bc0 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -15,6 +15,7 @@ */ import {google} from '../../protos/protos'; +import * as otel from '../opentelemetry-tracing'; /** * Strings are the only allowed values for keys and values in message attributes. @@ -24,7 +25,9 @@ export type Attributes = Record; /** * The basic {data, attributes} for a message to be published. */ -export interface PubsubMessage extends google.pubsub.v1.IPubsubMessage { +export interface PubsubMessage + extends google.pubsub.v1.IPubsubMessage, + otel.MessageWithAttributes { /** * If we've calculated the size of this message, it will be cached here. * This is done to avoid having to build up the attribute size over and over. diff --git a/src/subscriber.ts b/src/subscriber.ts index c7137ad78..b0023be57 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -18,7 +18,7 @@ import {DateStruct, PreciseDate} from '@google-cloud/precise-date'; import {replaceProjectIdToken} from '@google-cloud/projectify'; import {promisify} from '@google-cloud/promisify'; import {EventEmitter} from 'events'; -import {SpanContext, Span, SpanKind} from '@opentelemetry/api'; +import {Span} from '@opentelemetry/api'; import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; import {google} from '../protos/protos'; @@ -91,7 +91,7 @@ export class AckError extends Error { * }); * ``` */ -export class Message { +export class Message implements otel.MessageWithAttributes { ackId: string; attributes: {[key: string]: string}; data: Buffer; @@ -744,12 +744,7 @@ export class Subscriber extends EventEmitter { * @param {Message} message One of the received messages * @private */ - private _constructSpan(message: Message): Span | undefined { - // Handle cases where OpenTelemetry is disabled or no span context was sent through message - if (!this._useLegacyOpenTelemetry || !message.attributes) { - return undefined; - } - + private getParentSpan(message: Message): Span | undefined { const enabled = otel.isEnabled({ enableOpenTelemetryTracing: this._useLegacyOpenTelemetry, }); @@ -757,36 +752,7 @@ export class Subscriber extends EventEmitter { return undefined; } - if (!otel.containsSpanContext(message)) { - return undefined; - } - - const spanAttributes = { - // Original span attributes - ackId: message.ackId, - deliveryAttempt: message.deliveryAttempt, - // - // based on https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/trace/semantic_conventions/messaging.md#topic-with-multiple-consumers - [SemanticAttributes.MESSAGING_SYSTEM]: 'pubsub', - [SemanticAttributes.MESSAGING_OPERATION]: 'process', - [SemanticAttributes.MESSAGING_DESTINATION]: this.name, - [SemanticAttributes.MESSAGING_DESTINATION_KIND]: 'topic', - [SemanticAttributes.MESSAGING_MESSAGE_ID]: message.id, - [SemanticAttributes.MESSAGING_PROTOCOL]: 'pubsub', - [SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES]: ( - message.data as Buffer - ).length, - // Not in Opentelemetry semantic convention but mimics naming - 'messaging.pubsub.received_at': message.received, - 'messaging.pubsub.acknowlege_id': message.ackId, - 'messaging.pubsub.delivery_attempt': message.deliveryAttempt, - }; - - // Subscriber spans should always have a publisher span as a parent. - // Return undefined if no parent is provided - const spanName = `${this.name} process`; - - return otel.extractSpan(message, spanName, spanAttributes, enabled); + return otel.extractSpan(message, this.name, enabled); } /** @@ -816,7 +782,7 @@ export class Subscriber extends EventEmitter { for (const data of receivedMessages!) { const message = new Message(this, data); - const span: Span | undefined = this._constructSpan(message); + const span: Span | undefined = this.getParentSpan(message); if (this.isOpen) { message.modAck(this.ackDeadline); diff --git a/test/opentelemetry-tracing.ts b/test/opentelemetry-tracing.ts index 8784ad11b..a75d2dd95 100644 --- a/test/opentelemetry-tracing.ts +++ b/test/opentelemetry-tracing.ts @@ -17,36 +17,23 @@ import * as assert from 'assert'; import {describe, it, beforeEach} from 'mocha'; -import * as api from '@opentelemetry/api'; import * as trace from '@opentelemetry/tracing'; import * as otel from '../src/opentelemetry-tracing'; import {exporter} from './tracing'; import {SpanKind} from '@opentelemetry/api'; import sinon = require('sinon'); +import {PubsubMessage} from '../src/publisher'; describe('OpenTelemetryTracer', () => { - let span: trace.Span; - const spanName = 'test-span'; - const spanContext: api.SpanContext = { - traceId: 'd4cda95b652f4a1592b449d5929fda1b', - spanId: '6e0c63257de34c92', - traceFlags: api.TraceFlags.SAMPLED, - }; - const spanAttributes: otel.SpanAttributes = { - foo: 'bar', - }; - const context = otel.spanContextToContext(spanContext); - beforeEach(() => { exporter.reset(); }); it('creates a span', () => { - span = otel.createSpan( - spanName, - SpanKind.PRODUCER, - spanAttributes, - context + const message: PubsubMessage = {}; + const span = otel.SpanMaker.createPublisherSpan( + message, + 'test topic' ) as trace.Span; span.end(); @@ -54,23 +41,18 @@ describe('OpenTelemetryTracer', () => { assert.notStrictEqual(spans.length, 0); const exportedSpan = spans.concat().pop()!; - assert.strictEqual(exportedSpan.name, spanName); - assert.deepStrictEqual(exportedSpan.attributes, spanAttributes); - assert.strictEqual(exportedSpan.parentSpanId, spanContext.spanId); + assert.strictEqual(exportedSpan.name, 'test topic send'); assert.strictEqual(exportedSpan.kind, SpanKind.PRODUCER); }); it('injects a trace context', () => { - span = otel.createSpan( - spanName, - SpanKind.PRODUCER, - spanAttributes, - context - ) as trace.Span; - - const message = { + const message: PubsubMessage = { attributes: {}, }; + const span = otel.SpanMaker.createPublisherSpan( + message, + 'test topic' + ) as trace.Span; otel.injectSpan(span, message, otel.OpenTelemetryLevel.Modern); @@ -83,18 +65,13 @@ describe('OpenTelemetryTracer', () => { }); it('injects a trace context and legacy baggage', () => { - span = otel.createSpan( - spanName, - SpanKind.PRODUCER, - spanAttributes, - context - ) as trace.Span; - - const message = { + const message: PubsubMessage = { attributes: {}, }; + const span = otel.SpanMaker.createPublisherSpan(message, 'test topic'); otel.injectSpan(span, message, otel.OpenTelemetryLevel.Legacy); + assert.strictEqual( Object.getOwnPropertyNames(message.attributes).includes( otel.modernAttributeName @@ -110,21 +87,16 @@ describe('OpenTelemetryTracer', () => { }); it('should issue a warning if OpenTelemetry span context key is set', () => { - span = otel.createSpan( - spanName, - SpanKind.PRODUCER, - spanAttributes, - context - ) as trace.Span; + const message: PubsubMessage = { + attributes: { + [otel.legacyAttributeName]: 'foobar', + [otel.modernAttributeName]: 'bazbar', + }, + }; + const span = otel.SpanMaker.createPublisherSpan(message, 'test topic'); const warnSpy = sinon.spy(console, 'warn'); try { - const message = { - attributes: { - [otel.legacyAttributeName]: 'foobar', - [otel.modernAttributeName]: 'bazbar', - }, - }; otel.injectSpan(span, message, otel.OpenTelemetryLevel.Legacy); assert.strictEqual(warnSpy.callCount, 2); } finally { @@ -152,13 +124,6 @@ describe('OpenTelemetryTracer', () => { }); it('extracts a trace context', () => { - span = otel.createSpan( - spanName, - SpanKind.PRODUCER, - spanAttributes, - context - ) as trace.Span; - const message = { attributes: { [otel.modernAttributeName]: @@ -168,8 +133,7 @@ describe('OpenTelemetryTracer', () => { const childSpan = otel.extractSpan( message, - 'child', - {}, + 'test sub', otel.OpenTelemetryLevel.Modern ); assert.strictEqual( diff --git a/test/subscriber.ts b/test/subscriber.ts index 1bed1e346..2d8b6dc79 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -32,7 +32,6 @@ import {MessageStreamOptions} from '../src/message-stream'; import * as s from '../src/subscriber'; import {Subscription} from '../src/subscription'; import {SpanKind} from '@opentelemetry/api'; -import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; import {Duration} from '../src'; const stubs = new Map(); @@ -649,6 +648,11 @@ describe('Subscriber', () => { const inventory: FakeLeaseManager = stubs.get('inventory'); const addStub = sandbox.stub(inventory, 'add').callsFake(() => { const [addMsg] = addStub.lastCall.args; + + // OTel is enabled during tests, so we need to delete the baggage. + // eslint-disable-next-line @typescript-eslint/no-explicit-any + delete (addMsg as any).telemetrySpan; + assert.deepStrictEqual(addMsg, message); // test for receipt @@ -794,7 +798,7 @@ describe('Subscriber', () => { assert.strictEqual(firstSpan.parentSpanId, parentSpanContext.spanId); assert.strictEqual( firstSpan.name, - `${subscriber.name} process`, + `${subscriber.name} receive`, 'name of span should match' ); assert.strictEqual( @@ -802,32 +806,9 @@ describe('Subscriber', () => { SpanKind.CONSUMER, 'span kind should be CONSUMER' ); - assert.strictEqual( - firstSpan.attributes[SemanticAttributes.MESSAGING_OPERATION], - 'process', - 'span messaging operation should match' - ); - assert.strictEqual( - firstSpan.attributes[SemanticAttributes.MESSAGING_SYSTEM], - 'pubsub' - ); - assert.strictEqual( - firstSpan.attributes[SemanticAttributes.MESSAGING_MESSAGE_ID], - messageWithSpanContext.message.messageId, - 'span messaging id should match' - ); - assert.strictEqual( - firstSpan.attributes[SemanticAttributes.MESSAGING_DESTINATION], - subscriber.name, - 'span messaging destination should match' - ); - assert.strictEqual( - firstSpan.attributes[SemanticAttributes.MESSAGING_DESTINATION_KIND], - 'topic' - ); }); - it('does not export a span when a span context is not present on message', () => { + it('exports a span even when a span context is not present on message', () => { subscriber = new Subscriber(subscription, enableTracing); const pullResponse: s.PullResponse = { @@ -837,7 +818,7 @@ describe('Subscriber', () => { // Receive message and assert that it was exported const stream: FakeMessageStream = stubs.get('messageStream'); stream.emit('data', pullResponse); - assert.strictEqual(exporter.getFinishedSpans().length, 0); + assert.strictEqual(exporter.getFinishedSpans().length, 1); }); }); From cbd5992da0710e6510dda6ea47fb59cb1faec905 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Mon, 28 Nov 2022 17:42:24 -0500 Subject: [PATCH 04/52] add more spans --- src/lease-manager.ts | 16 ++++- src/opentelemetry-tracing.ts | 14 ++-- src/publisher/flow-publisher.ts | 5 ++ src/publisher/message-queues.ts | 13 ++++ src/publisher/pubsub-message.ts | 17 +++++ src/subscriber.ts | 123 ++++++++++++++++++++++++++++---- test/lease-manager.ts | 11 +++ test/subscriber.ts | 21 +++++- 8 files changed, 201 insertions(+), 19 deletions(-) diff --git a/src/lease-manager.ts b/src/lease-manager.ts index b616e0ce2..e52ee8601 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -17,6 +17,7 @@ import {EventEmitter} from 'events'; import {Message, Subscriber} from './subscriber'; import {defaultOptions} from './default-options'; +import * as otel from './opentelemetry-tracing'; export interface FlowControlOptions { allowExcessMessages?: boolean; @@ -104,6 +105,10 @@ export class LeaseManager extends EventEmitter { this._messages.add(message); this.bytes += message.length; + // TODO: How do these two related in Node? + message.telemetrySub.schedulerStart(); + message.telemetrySub.flowStart(); + if (allowExcessMessages! || !wasFull) { this._dispense(message); } else { @@ -240,7 +245,16 @@ export class LeaseManager extends EventEmitter { */ private _dispense(message: Message): void { if (this._subscriber.isOpen) { - process.nextTick(() => this._subscriber.emit('message', message)); + message.telemetrySub.flowEnd(); + message.telemetrySub.schedulerEnd(); + process.nextTick(() => { + const span = otel.SpanMaker.createReceiveProcessSpan( + message, + this._subscriber.name + ); + this._subscriber.emit('message', message); + span?.end(); + }); } } /** diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index bb2ad031d..5406a6628 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -31,6 +31,8 @@ import {PublishOptions} from './publisher/index'; import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; import {Duration} from './temporal'; +export {Span}; + // We need this to get the library version. // eslint-disable-next-line @typescript-eslint/no-var-requires const packageJson = require('../../package.json'); @@ -285,22 +287,24 @@ export class SpanMaker { return SpanMaker.createChildSpan(message, 'subscriber flow control'); } - static createSchedulerSpan(message: MessageWithAttributes): Span | undefined { + static createReceiveSchedulerSpan( + message: MessageWithAttributes + ): Span | undefined { return SpanMaker.createChildSpan(message, 'subscribe scheduler'); } - static createProcessSpan( + static createReceiveProcessSpan( message: MessageWithAttributes, subName: string ): Span | undefined { return SpanMaker.createChildSpan(message, `${subName} process`); } - static setProcessResult(span: Span, isAck: boolean) { + static setReceiveProcessResult(span: Span, isAck: boolean) { span.setAttribute('messaging.pubsub.result', isAck ? 'ack' : 'nack'); } - static createLeaseSpan( + static createReceiveLeaseSpan( message: MessageWithAttributes, deadline: Duration, isInitial: boolean @@ -314,7 +318,7 @@ export class SpanMaker { return span; } - static createResponseSpan( + static createReceiveResponseSpan( message: MessageWithAttributes, isAck: boolean ): Span | undefined { diff --git a/src/publisher/flow-publisher.ts b/src/publisher/flow-publisher.ts index ebfee5b0f..adfffcb05 100644 --- a/src/publisher/flow-publisher.ts +++ b/src/publisher/flow-publisher.ts @@ -17,6 +17,7 @@ import {Publisher} from '.'; import {FlowControl} from './flow-control'; import {PubsubMessage, calculateMessageSize} from './pubsub-message'; +import * as otel from '../opentelemetry-tracing'; /** * Encapsulates a series of message publishes from a rapid loop (or similar @@ -76,7 +77,11 @@ export class FlowControlledPublisher { * ``` */ publish(message: PubsubMessage): Promise | null { + const flowSpan = message.telemetrySpan + ? otel.SpanMaker.createPublishFlowSpan(message) + : undefined; const doPublish = () => { + flowSpan?.end(); this.doPublish(message); }; diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 4d367b8c4..312c3a504 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -21,6 +21,7 @@ import {BatchPublishOptions, MessageBatch} from './message-batch'; import {PublishError} from './publish-error'; import {Publisher, PubsubMessage, PublishCallback} from './'; import {google} from '../../protos/protos'; +import * as otel from '../opentelemetry-tracing'; export interface PublishDone { (err: ServiceError | null): void; @@ -96,6 +97,10 @@ export abstract class MessageQueue extends EventEmitter { return; } + messages.forEach(m => { + m.telemetryRpc = otel.SpanMaker.createPublishRpcSpan(m); + }); + topic.request( { client: 'PublisherClient', @@ -104,6 +109,10 @@ export abstract class MessageQueue extends EventEmitter { gaxOpts: settings.gaxOpts!, }, (err, resp) => { + messages.forEach(m => { + m.telemetryRpc?.end(); + }); + const messageIds = (resp && resp.messageIds) || []; callbacks.forEach((callback, i) => callback(err, messageIds[i])); @@ -147,6 +156,8 @@ export class Queue extends MessageQueue { this.publish(); } + message.telemetryBatching = otel.SpanMaker.createPublishBatchSpan(message); + this.batch.add(message, callback); if (this.batch.isFull()) { @@ -172,6 +183,8 @@ export class Queue extends MessageQueue { delete this.pending; } + messages.forEach(m => m.telemetryBatching?.end()); + this._publish(messages, callbacks, (err: null | ServiceError) => { if (err) { definedCallback(err); diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index 269967bc0..b49389191 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -38,6 +38,23 @@ export interface PubsubMessage * @private */ calculatedSize?: number; + + // The following are here instead of inside an object (like subs) because we + // don't get to control what these objects are. They come from grpc. + + /** + * If telemetry is enabled, track the batch span. + * + * @private + */ + telemetryBatching?: otel.Span; + + /** + * If telemetry is enabled, track the RPC send time span. + * + * @private + */ + telemetryRpc?: otel.Span; } /** diff --git a/src/subscriber.ts b/src/subscriber.ts index b0023be57..6d2429e0b 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -18,8 +18,6 @@ import {DateStruct, PreciseDate} from '@google-cloud/precise-date'; import {replaceProjectIdToken} from '@google-cloud/projectify'; import {promisify} from '@google-cloud/promisify'; import {EventEmitter} from 'events'; -import {Span} from '@opentelemetry/api'; -import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; import {google} from '../protos/protos'; import {Histogram} from './histogram'; @@ -64,6 +62,67 @@ export class AckError extends Error { } } +/** + * Tracks the various spans in receive telemetry. + * + * @private + */ +export class SubscriberTelemetry { + parent: Message; + sub: Subscriber; + + constructor(parent: Message, sub: Subscriber) { + this.parent = parent; + this.sub = sub; + } + + flowStart() { + if (!this.flow) { + this.flow = otel.SpanMaker.createReceiveFlowSpan(this.parent); + } + } + + flowEnd() { + if (this.flow) { + this.flow.end(); + this.flow = undefined; + } + } + + schedulerStart() { + if (!this.scheduler) { + this.scheduler = otel.SpanMaker.createReceiveSchedulerSpan(this.parent); + } + } + + schedulerEnd() { + if (this.scheduler) { + this.scheduler.end(); + this.scheduler = undefined; + } + } + + processingStart(subName: string) { + if (!this.processing) { + this.processing = otel.SpanMaker.createReceiveProcessSpan( + this.parent, + subName + ); + } + } + + processingEnd() { + if (this.processing) { + this.processing.end(); + this.processing = undefined; + } + } + + private flow?: otel.Span; + private scheduler?: otel.Span; + private processing?: otel.Span; +} + /** * Date object with nanosecond precision. Supports all standard Date arguments * in addition to several custom types. @@ -103,6 +162,26 @@ export class Message implements otel.MessageWithAttributes { private _handled: boolean; private _length: number; private _subscriber: Subscriber; + + /** + * @private + * + * Tracks any telemetry span through the library, on the receive side. This will + * be the original publisher-side span if we have one. + * + * This needs to be declared explicitly here, because having a public class + * implement a private interface seems to confuse TypeScript. (And it's needed + * in unit tests.) + */ + telemetrySpan?: otel.Span; + + /** + * @private + * + * Tracks subscriber-specific telemetry spans through the library. + */ + telemetrySub: SubscriberTelemetry; + /** * @hideconstructor * @@ -180,6 +259,13 @@ export class Message implements otel.MessageWithAttributes { */ this.received = Date.now(); + /** + * Telemetry tracking objects. + * + * @private + */ + this.telemetrySub = new SubscriberTelemetry(this, sub); + this._handled = false; this._length = this.data.length; this._subscriber = sub; @@ -528,6 +614,8 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); + const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, true); + // Ignore this in this version of the method (but hook then/catch // to avoid unhandled exceptions). const resultPromise = this._acks.add(message); @@ -535,6 +623,9 @@ export class Subscriber extends EventEmitter { resultPromise.catch(() => {}); await this._acks.onFlush(); + + ackSpan?.end(); + this._inventory.remove(message); } @@ -550,7 +641,12 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); + const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, true); + await this._acks.add(message); + + ackSpan?.end(); + this._inventory.remove(message); // No exception means Success. @@ -650,7 +746,12 @@ export class Subscriber extends EventEmitter { * @private */ async nack(message: Message): Promise { + const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, false); + await this.modAck(message, 0); + + ackSpan?.end(); + this._inventory.remove(message); } @@ -664,7 +765,10 @@ export class Subscriber extends EventEmitter { * @private */ async nackWithResponse(message: Message): Promise { - return await this.modAckWithResponse(message, 0); + const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, false); + const response = await this.modAckWithResponse(message, 0); + ackSpan?.end(); + return response; } /** @@ -744,15 +848,13 @@ export class Subscriber extends EventEmitter { * @param {Message} message One of the received messages * @private */ - private getParentSpan(message: Message): Span | undefined { + private createParentSpan(message: Message): void { const enabled = otel.isEnabled({ enableOpenTelemetryTracing: this._useLegacyOpenTelemetry, }); - if (!enabled) { - return undefined; + if (enabled) { + otel.extractSpan(message, this.name, enabled); } - - return otel.extractSpan(message, this.name, enabled); } /** @@ -782,7 +884,7 @@ export class Subscriber extends EventEmitter { for (const data of receivedMessages!) { const message = new Message(this, data); - const span: Span | undefined = this.getParentSpan(message); + this.createParentSpan(message); if (this.isOpen) { message.modAck(this.ackDeadline); @@ -790,9 +892,6 @@ export class Subscriber extends EventEmitter { } else { message.nack(); } - if (span) { - span.end(); - } } } diff --git a/test/lease-manager.ts b/test/lease-manager.ts index cf8def1a6..6df782a63 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -36,9 +36,20 @@ class FakeSubscriber extends EventEmitter { async modAck(): Promise {} } +class FakeSubscriberTelemetry { + flowStart() {} + flowEnd() {} + schedulerStart() {} + schedulerEnd() {} + processingStart(subName: string) {} + processingEnd() {} +} + class FakeMessage { length = 20; received: number; + telemetrySub: FakeSubscriberTelemetry = new FakeSubscriberTelemetry(); + constructor() { this.received = Date.now(); } diff --git a/test/subscriber.ts b/test/subscriber.ts index 2d8b6dc79..a8ecc0e13 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -26,7 +26,7 @@ import * as uuid from 'uuid'; import * as opentelemetry from '@opentelemetry/api'; import {HistogramOptions} from '../src/histogram'; -import {FlowControlOptions} from '../src/lease-manager'; +import {FlowControlOptions, LeaseManager} from '../src/lease-manager'; import {BatchOptions} from '../src/message-queues'; import {MessageStreamOptions} from '../src/message-stream'; import * as s from '../src/subscriber'; @@ -59,6 +59,10 @@ class FakeSubscription { pubsub = new FakePubSub(); } +interface PublicInventory { + _inventory: LeaseManager; +} + class FakeHistogram { options?: HistogramOptions; constructor(options?: HistogramOptions) { @@ -787,10 +791,17 @@ describe('Subscriber', () => { receivedMessages: [messageWithSpanContext], }; + const openedSub = subscriber as unknown as PublicInventory; + sandbox.stub(openedSub._inventory, 'add').callsFake((m: s.Message) => { + message = m; + }); + // Receive message and assert that it was exported const msgStream = stubs.get('messageStream'); msgStream.emit('data', pullResponse); + message.telemetrySpan?.end(); + const spans = exporter.getFinishedSpans(); assert.strictEqual(spans.length, 1); const firstSpan = spans.concat().shift(); @@ -810,14 +821,22 @@ describe('Subscriber', () => { it('exports a span even when a span context is not present on message', () => { subscriber = new Subscriber(subscription, enableTracing); + subscriber.open(); const pullResponse: s.PullResponse = { receivedMessages: [RECEIVED_MESSAGE], }; + const openedSub = subscriber as unknown as PublicInventory; + sandbox.stub(openedSub._inventory, 'add').callsFake((m: s.Message) => { + message = m; + }); + // Receive message and assert that it was exported const stream: FakeMessageStream = stubs.get('messageStream'); stream.emit('data', pullResponse); + + message.telemetrySpan?.end(); assert.strictEqual(exporter.getFinishedSpans().length, 1); }); }); From 75acbaba003d89fc046615224e9024543fca9e3c Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 30 Nov 2022 15:40:46 -0500 Subject: [PATCH 05/52] wrap subscriber on() handlers to get better receive process spans --- src/lease-manager.ts | 9 --- src/subscriber.ts | 37 +++++++++- src/wrapping-emitter.ts | 145 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 180 insertions(+), 11 deletions(-) create mode 100644 src/wrapping-emitter.ts diff --git a/src/lease-manager.ts b/src/lease-manager.ts index e52ee8601..2c028997e 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -17,7 +17,6 @@ import {EventEmitter} from 'events'; import {Message, Subscriber} from './subscriber'; import {defaultOptions} from './default-options'; -import * as otel from './opentelemetry-tracing'; export interface FlowControlOptions { allowExcessMessages?: boolean; @@ -105,8 +104,6 @@ export class LeaseManager extends EventEmitter { this._messages.add(message); this.bytes += message.length; - // TODO: How do these two related in Node? - message.telemetrySub.schedulerStart(); message.telemetrySub.flowStart(); if (allowExcessMessages! || !wasFull) { @@ -246,14 +243,8 @@ export class LeaseManager extends EventEmitter { private _dispense(message: Message): void { if (this._subscriber.isOpen) { message.telemetrySub.flowEnd(); - message.telemetrySub.schedulerEnd(); process.nextTick(() => { - const span = otel.SpanMaker.createReceiveProcessSpan( - message, - this._subscriber.name - ); this._subscriber.emit('message', message); - span?.end(); }); } } diff --git a/src/subscriber.ts b/src/subscriber.ts index 6d2429e0b..71ccd695a 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -17,7 +17,6 @@ import {DateStruct, PreciseDate} from '@google-cloud/precise-date'; import {replaceProjectIdToken} from '@google-cloud/projectify'; import {promisify} from '@google-cloud/promisify'; -import {EventEmitter} from 'events'; import {google} from '../protos/protos'; import {Histogram} from './histogram'; @@ -29,6 +28,7 @@ import {defaultOptions} from './default-options'; import {SubscriberClient} from './v1'; import * as otel from './opentelemetry-tracing'; import {Duration} from './temporal'; +import {EmitterCallback, WrappingEmitter} from './wrapping-emitter'; export type PullResponse = google.pubsub.v1.IStreamingPullResponse; export type SubscriptionProperties = @@ -443,7 +443,7 @@ const minAckDeadlineForExactlyOnceDelivery = Duration.from({seconds: 60}); * @param {Subscription} subscription The corresponding subscription. * @param {SubscriberOptions} options The subscriber options. */ -export class Subscriber extends EventEmitter { +export class Subscriber extends WrappingEmitter { ackDeadline: number; maxMessages: number; maxBytes: number; @@ -464,6 +464,7 @@ export class Subscriber extends EventEmitter { constructor(subscription: Subscription, options = {}) { super(); + this.setEmitterWrapper(this.listenerWrapper.bind(this)); this.ackDeadline = defaultOptions.subscription.ackDeadline; this.maxMessages = defaultOptions.subscription.maxOutstandingMessages; @@ -478,6 +479,38 @@ export class Subscriber extends EventEmitter { this.setOptions(options); } + /** + * This wrapper will be called as part of the emit() process. This lets + * us capture the full time span of processing even if the user is using + * async callbacks. + * + * @private + */ + private listenerWrapper( + eventName: string | symbol, + listener: EmitterCallback, + args: unknown[] + ) { + if (eventName !== 'message') { + return listener(...args); + } else { + const span = otel.SpanMaker.createReceiveProcessSpan( + args[0] as Message, + this.name + ); + + // If the user returned a Promise, that means they used an async handler. + // In that case, we need to tag on to their Promise to end the span. + // Otherwise, the listener chain is sync, and we can close out sync. + const result = listener(...args) as unknown as Promise; + if (!!result && typeof result.then === 'function') { + result.then(() => span?.end()); + } else { + span?.end(); + } + } + } + /** * Update our ack extension time that will be used by the lease manager * for sending modAcks. diff --git a/src/wrapping-emitter.ts b/src/wrapping-emitter.ts new file mode 100644 index 000000000..2bbd8c370 --- /dev/null +++ b/src/wrapping-emitter.ts @@ -0,0 +1,145 @@ +// Copyright 2022 Google LLC +// +// 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 +// +// 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. + +import {EventEmitter} from 'stream'; + +/** + * TypeScript alias for the built-in emitter listener callback. + * + * @private + */ +export interface EmitterCallback { + // This must match the Node built-in type. + // eslint-disable-next-line @typescript-eslint/no-explicit-any + (...args: any[]): void; +} + +/** + * Listener wrapper function type - we'll pass the event name, the original + * user callback, and any args that came with the emit. + * + * @private + */ +export interface Wrapper { + ( + eventName: string | symbol, + callback: EmitterCallback, + // Again, matching built-in types. + // eslint-disable-next-line @typescript-eslint/no-explicit-any + args: any[] + ): Promise | void; +} + +/** + * Subclass for the standard EventEmitter that lets you wrap user listen + * handlers. In this library's particular case, this is for the purpose of + * being able to properly wrap these callbacks in OTel spans even across + * async handler callbacks. + * + * This might be overkill for this use case, but the hope is to avoid + * breaking users on a minor version change. + * + * @private + */ +export class WrappingEmitter extends EventEmitter { + private wrapper: Wrapper; + private mapper = new Map>(); + + /** + * Pass a wrapper function here, or a default pass-through will be used. + * + * @private + */ + constructor(wrapper?: Wrapper) { + super(); + this.wrapper = + wrapper ?? + ((event, cb, args) => { + cb(...args); + }); + } + + /** + * Manually set a wrapper pass-through. Because this might be exported to + * anyone using this class, the name is a bit extra verbose. + * + * @private + */ + setEmitterWrapper(wrapper: Wrapper) { + this.wrapper = wrapper; + } + + // These two are just wrappers for addListener/removeListener. + on(eventName: string | symbol, listener: EmitterCallback): this { + return this.addListener(eventName, listener); + } + + off(eventName: string | symbol, listener: EmitterCallback): this { + return this.removeListener(eventName, listener); + } + + // This addListener wrapper will create a one-off, unique wrapper function + // to pass down into super.addListener, and save a mapping of it for later removal. + addListener(eventName: string | symbol, listener: EmitterCallback): this { + const getMapper = () => { + return (...args: unknown[]) => { + return this.wrapper(eventName, listener, args); + }; + }; + const newListener = getMapper(); + const subset = + this.mapper.get(eventName) ?? new Map(); + subset.set(listener, newListener); + this.mapper.set(eventName, subset); + super.addListener(eventName, newListener); + + return this; + } + + // This removeListener wrapper translates the user-passed handler back into + // the unique wrapper function, and then passes that down to super.removeListener. + // This also tries to keep a more or less clean listener mapping list. + removeListener(eventName: string | symbol, listener: EmitterCallback): this { + let listenerToRemove = listener; + + const subset = this.mapper.get(eventName); + if (subset) { + const wrapper = subset.get(listener); + if (wrapper) { + listenerToRemove = wrapper; + } + subset.delete(listener); + if (!subset.size) { + this.mapper.delete(eventName); + } + } + + super.removeListener(eventName, listenerToRemove); + + return this; + } + + // Wrapper for removeAllListeners that also deletes any mappings we had for the event. + removeAllListeners(event?: string | symbol | undefined): this { + if (event) { + this.mapper.delete(event); + } else { + this.mapper.clear(); + } + + super.removeAllListeners(event); + + return this; + } +} From bb6cf85a8f84142b6255086e27ed7444f5fdcc60 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 1 Dec 2022 15:50:44 -0500 Subject: [PATCH 06/52] update otel core version for tests --- package.json | 1 + 1 file changed, 1 insertion(+) diff --git a/package.json b/package.json index 7f091f2dd..83e454ddc 100644 --- a/package.json +++ b/package.json @@ -66,6 +66,7 @@ }, "devDependencies": { "@grpc/proto-loader": "^0.7.0", + "@opentelemetry/core": "^1.8.0", "@opentelemetry/tracing": "^0.24.0", "@types/execa": "^0.9.0", "@types/extend": "^3.0.0", From 5cddaedb72db9e7ca9a333c2a5d5983b23cc4e72 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 8 Dec 2022 14:56:27 -0500 Subject: [PATCH 07/52] fix: change some span names for doc updates --- src/opentelemetry-tracing.ts | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index 5406a6628..a499c7790 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -278,7 +278,7 @@ export class SpanMaker { } static createPublishRpcSpan(message: PubsubMessage): Span | undefined { - return SpanMaker.createChildSpan(message, 'send Publish'); + return SpanMaker.createChildSpan(message, 'publish'); } static createReceiveFlowSpan( @@ -309,7 +309,7 @@ export class SpanMaker { deadline: Duration, isInitial: boolean ): Span | undefined { - const span = SpanMaker.createChildSpan(message, 'send ModifyAckDeadline'); + const span = SpanMaker.createChildSpan(message, 'modify ack deadline'); span?.setAttribute( 'messaging.pubsub.modack_deadline_seconds', deadline.totalOf('second') @@ -322,9 +322,7 @@ export class SpanMaker { message: MessageWithAttributes, isAck: boolean ): Span | undefined { - const name = isAck - ? 'send Acknowledgement' - : 'send Negative Acknowledgement'; + const name = isAck ? 'ack' : 'nack'; return SpanMaker.createChildSpan(message, name); } } From d523b1fd6d241b97672da31421f4984a66734c55 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 8 Dec 2022 15:51:28 -0500 Subject: [PATCH 08/52] fix: move emitter wrapper out one level, to catch user functions properly --- src/subscriber.ts | 37 ++----------------------------------- src/subscription.ts | 42 +++++++++++++++++++++++++++++++++++++++--- 2 files changed, 41 insertions(+), 38 deletions(-) diff --git a/src/subscriber.ts b/src/subscriber.ts index 71ccd695a..4cd390982 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -28,7 +28,7 @@ import {defaultOptions} from './default-options'; import {SubscriberClient} from './v1'; import * as otel from './opentelemetry-tracing'; import {Duration} from './temporal'; -import {EmitterCallback, WrappingEmitter} from './wrapping-emitter'; +import {EventEmitter} from 'events'; export type PullResponse = google.pubsub.v1.IStreamingPullResponse; export type SubscriptionProperties = @@ -443,7 +443,7 @@ const minAckDeadlineForExactlyOnceDelivery = Duration.from({seconds: 60}); * @param {Subscription} subscription The corresponding subscription. * @param {SubscriberOptions} options The subscriber options. */ -export class Subscriber extends WrappingEmitter { +export class Subscriber extends EventEmitter { ackDeadline: number; maxMessages: number; maxBytes: number; @@ -464,7 +464,6 @@ export class Subscriber extends WrappingEmitter { constructor(subscription: Subscription, options = {}) { super(); - this.setEmitterWrapper(this.listenerWrapper.bind(this)); this.ackDeadline = defaultOptions.subscription.ackDeadline; this.maxMessages = defaultOptions.subscription.maxOutstandingMessages; @@ -479,38 +478,6 @@ export class Subscriber extends WrappingEmitter { this.setOptions(options); } - /** - * This wrapper will be called as part of the emit() process. This lets - * us capture the full time span of processing even if the user is using - * async callbacks. - * - * @private - */ - private listenerWrapper( - eventName: string | symbol, - listener: EmitterCallback, - args: unknown[] - ) { - if (eventName !== 'message') { - return listener(...args); - } else { - const span = otel.SpanMaker.createReceiveProcessSpan( - args[0] as Message, - this.name - ); - - // If the user returned a Promise, that means they used an async handler. - // In that case, we need to tag on to their Promise to end the span. - // Otherwise, the listener chain is sync, and we can close out sync. - const result = listener(...args) as unknown as Promise; - if (!!result && typeof result.then === 'function') { - result.then(() => span?.end()); - } else { - span?.end(); - } - } - } - /** * Update our ack extension time that will be used by the lease manager * for sending modAcks. diff --git a/src/subscription.ts b/src/subscription.ts index 29567ca12..031c1a175 100644 --- a/src/subscription.ts +++ b/src/subscription.ts @@ -14,7 +14,6 @@ * limitations under the License. */ -import {EventEmitter} from 'events'; import * as extend from 'extend'; import {CallOptions} from 'google-gax'; import snakeCase = require('lodash.snakecase'); @@ -42,12 +41,15 @@ import { SeekResponse, Snapshot, } from './snapshot'; -import {Subscriber, SubscriberOptions} from './subscriber'; +import {Message, Subscriber, SubscriberOptions} from './subscriber'; import {Topic} from './topic'; import {promisifySome} from './util'; export {AckError, AckResponse, AckResponses} from './subscriber'; +import {EmitterCallback, WrappingEmitter} from './wrapping-emitter'; +import * as otel from './opentelemetry-tracing'; + export type PushConfig = google.pubsub.v1.IPushConfig; export type OidcToken = google.pubsub.v1.PushConfig.IOidcToken; @@ -266,7 +268,7 @@ export type DetachSubscriptionResponse = EmptyResponse; * }); * ``` */ -export class Subscription extends EventEmitter { +export class Subscription extends WrappingEmitter { pubsub: PubSub; iam: IAM; name: string; @@ -277,6 +279,8 @@ export class Subscription extends EventEmitter { constructor(pubsub: PubSub, name: string, options?: SubscriptionOptions) { super(); + this.setEmitterWrapper(this.listenerWrapper.bind(this)); + options = options || {}; this.pubsub = pubsub; @@ -334,6 +338,38 @@ export class Subscription extends EventEmitter { this._listen(); } + /** + * This wrapper will be called as part of the emit() process. This lets + * us capture the full time span of processing even if the user is using + * async callbacks. + * + * @private + */ + private listenerWrapper( + eventName: string | symbol, + listener: EmitterCallback, + args: unknown[] + ) { + if (eventName !== 'message') { + return listener(...args); + } else { + const span = otel.SpanMaker.createReceiveProcessSpan( + args[0] as Message, + this.name + ); + + // If the user returned a Promise, that means they used an async handler. + // In that case, we need to tag on to their Promise to end the span. + // Otherwise, the listener chain is sync, and we can close out sync. + const result = listener(...args) as unknown as Promise; + if (!!result && typeof result.then === 'function') { + result.then(() => span?.end()); + } else { + span?.end(); + } + } + } + /** * Indicates if the Subscription is open and receiving messages. * From 197a21457dabb2f97956968102d768a261d07576 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 14 Dec 2022 18:05:49 -0500 Subject: [PATCH 09/52] fix: don't send anything besides what's actually needed during publish --- src/publisher/message-queues.ts | 3 ++- src/publisher/pubsub-message.ts | 12 ++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 312c3a504..e98602abc 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -22,6 +22,7 @@ import {PublishError} from './publish-error'; import {Publisher, PubsubMessage, PublishCallback} from './'; import {google} from '../../protos/protos'; import * as otel from '../opentelemetry-tracing'; +import {filterMessage} from './pubsub-message'; export interface PublishDone { (err: ServiceError | null): void; @@ -88,7 +89,7 @@ export abstract class MessageQueue extends EventEmitter { const {topic, settings} = this.publisher; const reqOpts = { topic: topic.name, - messages, + messages: messages.map(filterMessage), }; if (messages.length === 0) { if (typeof callback === 'function') { diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index b49389191..b6fe07ac2 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -57,6 +57,18 @@ export interface PubsubMessage telemetryRpc?: otel.Span; } +/** + * Since we tag a fair number of extra things into messages sent to the Pub/Sub + * server, this filters everything down to what needs to be sent. This should be + * used right before gRPC calls. + */ +export function filterMessage( + message: PubsubMessage +): google.pubsub.v1.IPubsubMessage { + const {data, attributes, messageId, publishTime, orderingKey} = message; + return {data, attributes, messageId, publishTime, orderingKey}; +} + /** * Precisely calculates the size of a message with optional `data` and * `attributes` fields. If a `data` field is present, its {@link Buffer#length} From a36cfcea8c1d70226203d2248b1415e2b3ddab75 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 14 Dec 2022 18:08:12 -0500 Subject: [PATCH 10/52] fix: close the publish span at RPC time, not up front --- src/publisher/index.ts | 4 ---- src/publisher/message-queues.ts | 1 + 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 66884a60d..a83d57019 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -228,10 +228,6 @@ export class Publisher { const queue = this.orderedQueues.get(key)!; queue.add(message, callback!); } - - if (span) { - span.end(); - } } /** diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index e98602abc..93a9feff6 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -112,6 +112,7 @@ export abstract class MessageQueue extends EventEmitter { (err, resp) => { messages.forEach(m => { m.telemetryRpc?.end(); + m.telemetrySpan?.end(); }); const messageIds = (resp && resp.messageIds) || []; From 5c4fef62f6be21a5a6bab7bd4fb098748d437535 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 14 Dec 2022 18:08:43 -0500 Subject: [PATCH 11/52] fix: delay caching of the trace provider so the user has a chance to make one --- src/opentelemetry-tracing.ts | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/src/opentelemetry-tracing.ts b/src/opentelemetry-tracing.ts index a499c7790..18fdb6f93 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/opentelemetry-tracing.ts @@ -42,10 +42,14 @@ const packageJson = require('../../package.json'); * * @private */ -const libraryTracer: Tracer = trace.getTracer( - '@google-cloud/pubsub', - packageJson.version -); +let cachedTracer: Tracer | undefined; +function getTracer(): Tracer { + const tracer = + cachedTracer ?? + trace.getTracer('@google-cloud/pubsub', packageJson.version); + cachedTracer = tracer; + return cachedTracer; +} /** * Determination of the level of OTel support we're providing. @@ -217,7 +221,7 @@ export class SpanMaker { 'messaging.pubsub.ordering_key': message.orderingKey, } as SpanAttributes; - const span: Span = libraryTracer.startSpan(`${topicName} send`, { + const span: Span = getTracer().startSpan(`${topicName} send`, { kind: SpanKind.PRODUCER, attributes: spanAttributes, }); @@ -235,7 +239,7 @@ export class SpanMaker { // Mostly we want to keep the context IDs; the attributes and such // are only something we do on the publish side. if (context) { - return libraryTracer.startSpan( + return getTracer().startSpan( name, { kind: SpanKind.CONSUMER, @@ -244,7 +248,7 @@ export class SpanMaker { parent ); } else { - return libraryTracer.startSpan(name, { + return getTracer().startSpan(name, { kind: SpanKind.CONSUMER, }); } @@ -256,7 +260,7 @@ export class SpanMaker { ): Span | undefined { const parent = message.telemetrySpan; if (parent) { - return libraryTracer.startSpan( + return getTracer().startSpan( name, { kind: SpanKind.INTERNAL, From b58557d849138c979c215cc22e2a62ab5c447985 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 15 Dec 2022 16:44:15 -0500 Subject: [PATCH 12/52] build: bring OTel versions back to compatible with Node 12 --- package.json | 6 +++--- samples/package.json | 7 +++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/package.json b/package.json index 83e454ddc..06646850c 100644 --- a/package.json +++ b/package.json @@ -52,7 +52,7 @@ "@google-cloud/projectify": "^3.0.0", "@google-cloud/promisify": "^2.0.0", "@opentelemetry/api": "^1.0.0", - "@opentelemetry/semantic-conventions": "~1.3.0", + "@opentelemetry/semantic-conventions": "~1.3.1", "@types/duplexify": "^3.6.0", "@types/long": "^4.0.0", "arrify": "^2.0.0", @@ -66,8 +66,8 @@ }, "devDependencies": { "@grpc/proto-loader": "^0.7.0", - "@opentelemetry/core": "^1.8.0", - "@opentelemetry/tracing": "^0.24.0", + "@opentelemetry/core": "~1.3.1", + "@opentelemetry/sdk-trace-base": "~1.3.1", "@types/execa": "^0.9.0", "@types/extend": "^3.0.0", "@types/lodash.snakecase": "^4.1.6", diff --git a/samples/package.json b/samples/package.json index 2ed6aff27..4c6c4e2ce 100644 --- a/samples/package.json +++ b/samples/package.json @@ -22,8 +22,11 @@ }, "dependencies": { "@google-cloud/pubsub": "^3.2.1", - "@opentelemetry/api": "^1.0.0", - "@opentelemetry/tracing": "^0.24.0", + "@opentelemetry/api": "^1.3.0", + "@opentelemetry/resources": "~1.3.1", + "@opentelemetry/sdk-node": "~1.3.1", + "@opentelemetry/sdk-trace-node": "~1.3.1", + "@opentelemetry/semantic-conventions": "~1.3.1", "avro-js": "^1.10.1", "p-defer": "^3.0.0", "protobufjs": "^7.0.0" From 4b7cf0c37c695fbd310a1aed3b0c08c7ea362455 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 15 Dec 2022 16:45:27 -0500 Subject: [PATCH 13/52] fix: clean up a few more namespace pollution issues in OTel support --- src/publisher/message-queues.ts | 5 ++++- src/publisher/pubsub-message.ts | 19 +++++++++++++++++-- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 93a9feff6..552471552 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -99,7 +99,10 @@ export abstract class MessageQueue extends EventEmitter { } messages.forEach(m => { - m.telemetryRpc = otel.SpanMaker.createPublishRpcSpan(m); + const span = otel.SpanMaker.createPublishRpcSpan(m); + if (span) { + m.telemetryRpc = span; + } }); topic.request( diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index b6fe07ac2..47a443835 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -65,8 +65,23 @@ export interface PubsubMessage export function filterMessage( message: PubsubMessage ): google.pubsub.v1.IPubsubMessage { - const {data, attributes, messageId, publishTime, orderingKey} = message; - return {data, attributes, messageId, publishTime, orderingKey}; + const filtered = {} as PubsubMessage; + if (message.data) { + filtered.data = message.data; + } + if (message.attributes) { + filtered.attributes = message.attributes; + } + if (message.messageId) { + filtered.messageId = message.messageId; + } + if (message.publishTime) { + filtered.publishTime = message.publishTime; + } + if (message.orderingKey) { + filtered.orderingKey = message.orderingKey; + } + return filtered; } /** From 45047c73076b04b90ea07d46186b608d1f53b444 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 15 Dec 2022 16:46:10 -0500 Subject: [PATCH 14/52] tests: manually end the publish span for tests --- test/publisher/index.ts | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 42d89602d..6e3453252 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -184,20 +184,22 @@ describe('Publisher', () => { describe('OpenTelemetry tracing', () => { let tracingPublisher: p.Publisher = {} as p.Publisher; - const enableTracing: p.PublishOptions = { - enableOpenTelemetryTracing: true, - }; const buffer = Buffer.from('Hello, world!'); beforeEach(() => { exporter.reset(); }); - it('export created spans', () => { + it('export created spans', async () => { // Setup trace exporting - tracingPublisher = new Publisher(topic, enableTracing); + tracingPublisher = new Publisher(topic); + const msg = {data: buffer} as p.PubsubMessage; + tracingPublisher.publishMessage(msg); + + // publishMessage is only the first part of the process now, + // so we need to manually end the span. + msg.telemetrySpan?.end(); - tracingPublisher.publishMessage({data: buffer}); const spans = exporter.getFinishedSpans(); assert.notStrictEqual(spans.length, 0, 'has span'); const createdSpan = spans.concat().pop()!; From 3995fd034ea315de798ac3e547a5339c57a6561d Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Thu, 15 Dec 2022 16:46:37 -0500 Subject: [PATCH 15/52] build: update package names for recent version updates --- test/opentelemetry-tracing.ts | 2 +- test/tracing.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/test/opentelemetry-tracing.ts b/test/opentelemetry-tracing.ts index a75d2dd95..bafd748d0 100644 --- a/test/opentelemetry-tracing.ts +++ b/test/opentelemetry-tracing.ts @@ -17,7 +17,7 @@ import * as assert from 'assert'; import {describe, it, beforeEach} from 'mocha'; -import * as trace from '@opentelemetry/tracing'; +import * as trace from '@opentelemetry/sdk-trace-base'; import * as otel from '../src/opentelemetry-tracing'; import {exporter} from './tracing'; import {SpanKind} from '@opentelemetry/api'; diff --git a/test/tracing.ts b/test/tracing.ts index 8b1b31146..7689253ad 100644 --- a/test/tracing.ts +++ b/test/tracing.ts @@ -18,7 +18,7 @@ import { BasicTracerProvider, InMemorySpanExporter, SimpleSpanProcessor, -} from '@opentelemetry/tracing'; +} from '@opentelemetry/sdk-trace-base'; /** * This file is used to initialise a global tracing provider and span exporter From f2f8f579e9d16281288b7ff785d140d5f494e3b5 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 16 Dec 2022 16:18:10 -0500 Subject: [PATCH 16/52] chore: get versions of OTel that work together and with Node 12 --- package.json | 2 +- samples/openTelemetryTracing.js | 225 ++++++++++----------- samples/package.json | 4 +- samples/typescript/openTelemetryTracing.ts | 128 ++++++++++++ 4 files changed, 242 insertions(+), 117 deletions(-) create mode 100644 samples/typescript/openTelemetryTracing.ts diff --git a/package.json b/package.json index 06646850c..e9f70f88c 100644 --- a/package.json +++ b/package.json @@ -51,7 +51,7 @@ "@google-cloud/precise-date": "^3.0.0", "@google-cloud/projectify": "^3.0.0", "@google-cloud/promisify": "^2.0.0", - "@opentelemetry/api": "^1.0.0", + "@opentelemetry/api": "~1.1.0", "@opentelemetry/semantic-conventions": "~1.3.1", "@types/duplexify": "^3.6.0", "@types/long": "^4.0.0", diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index 6ad4d0259..4e5e8a088 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -1,18 +1,20 @@ -/*! - * Copyright 2020 Google LLC - * - * 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 - * - * 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. - */ +// Copyright 2020-2022 Google LLC +// +// 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 +// +// 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. + +// This is a generated sample, using the typeless sample bot. Please +// look for the source TypeScript sample (.ts) for modifications. +'use strict'; /** * This sample demonstrates how to add OpenTelemetry tracing to the @@ -22,8 +24,6 @@ * at https://cloud.google.com/pubsub/docs. */ -'use strict'; - // sample-metadata: // title: OpenTelemetry Tracing // description: Demonstrates how to enable OpenTelemetry tracing in @@ -32,104 +32,101 @@ const SUBSCRIBER_TIMEOUT = 10; -function main( - topicNameOrId = 'YOUR_TOPIC_NAME_OR_ID', - subscriptionNameOrId = 'YOUR_SUBSCRIPTION_NAME_OR_ID', - data = 'Hello, world!' -) { - // [START opentelemetry_tracing] - /** - * TODO(developer): Uncomment these variables before running the sample. - */ - // const topicNameOrId = 'YOUR_TOPIC_OR_ID'; - // const subscriptionNameOrId = 'YOUR_SUBSCRIPTION_OR_ID'; - // const data = 'Hello, world!"; - - // Imports the Google Cloud client library - const {PubSub} = require('@google-cloud/pubsub'); - - // Imports the OpenTelemetry API - const openTelemetryApi = require('@opentelemetry/api'); - - // Imports the OpenTelemetry span handlers and exporter - const { - SimpleSpanProcessor, - BasicTracerProvider, - ConsoleSpanExporter, - } = require('@opentelemetry/tracing'); - - // Set up span processing and specify the console as the span exporter - const provider = new BasicTracerProvider(); - const exporter = new ConsoleSpanExporter(); - provider.addSpanProcessor(new SimpleSpanProcessor(exporter)); - - // Enable the diagnostic logger for OpenTelemetry - openTelemetryApi.diag.setLogger( - new openTelemetryApi.DiagConsoleLogger(), - openTelemetryApi.DiagLogLevel.INFO - ); - - provider.register(); - - // OpenTelemetry tracing is an optional feature and can be enabled by setting - // enableOpenTelemetryTracing as a publisher or subscriber option - const enableOpenTelemetryTracing = { - enableOpenTelemetryTracing: true, +// [START opentelemetry_tracing] +/** + * TODO(developer): Uncomment these variables before running the sample. + */ +// const topicNameOrId = 'YOUR_TOPIC_OR_ID'; +// const subscriptionNameOrId = 'YOUR_SUBSCRIPTION_OR_ID'; +// const data = 'Hello, world!"; + +// Imports the Google Cloud client library +const { PubSub } = require("@google-cloud/pubsub"); + +// Imports the OpenTelemetry API +const otel = require("@opentelemetry/sdk-trace-node"); +const { diag, DiagConsoleLogger, DiagLogLevel } = require("@opentelemetry/api"); +const { NodeTracerProvider } = otel; +const { SimpleSpanProcessor, ConsoleSpanExporter } = require("@opentelemetry/sdk-trace-base"); + + + + +const { Resource } = require("@opentelemetry/resources"); +const { SemanticResourceAttributes } = require("@opentelemetry/semantic-conventions"); + +// Enable the diagnostic logger for OpenTelemetry +diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); + +const exporter = new ConsoleSpanExporter(); + +const provider = new NodeTracerProvider({ + resource: new Resource({ + [SemanticResourceAttributes.SERVICE_NAME]: 'otel example' + }) +}); +const processor = new SimpleSpanProcessor(exporter); +provider.addSpanProcessor(processor); +provider.register(); + +// Creates a client; cache this for further use. +const pubSubClient = new PubSub(); + +async function publishMessage(topicNameOrId, data) { + // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) + const dataBuffer = Buffer.from(data); + const messageId = await pubSubClient. + topic(topicNameOrId). + publishMessage({ data: dataBuffer }); + console.log(`Message ${messageId} published.`); +} + +async function subscriptionListen(subscriptionNameOrId) { + // Message handler for subscriber + const messageHandler = (message) => { + console.log(`Message ${message.id} received.`); + message.ack(); + + // Ensure that all spans got flushed by the exporter + console.log('Cleaning up OpenTelemetry exporter...'); + exporter.shutdown().then(() => { + // Cleaned up exporter. + process.exit(0); + }); }; - // Creates a client; cache this for further use. - const pubSubClient = new PubSub(); - - async function publishMessage() { - // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) - const dataBuffer = Buffer.from(data); - const messageId = await pubSubClient - .topic(topicNameOrId, enableOpenTelemetryTracing) - .publishMessage({data: dataBuffer}); - console.log(`Message ${messageId} published.`); - } - - async function subscriptionListen() { - // Message handler for subscriber - const messageHandler = message => { - console.log(`Message ${message.id} received.`); - message.ack(); - - // Ensure that all spans got flushed by the exporter - console.log('Cleaning up Opentelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); - }; - - const errorHandler = error => { - console.log('Received error:', error); - - console.log('Cleaning up Opentelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); - }; - - // Listens for new messages from the topic - pubSubClient - .subscription(subscriptionNameOrId, enableOpenTelemetryTracing) - .on('message', messageHandler); - pubSubClient - .subscription(subscriptionNameOrId, enableOpenTelemetryTracing) - .on('error', errorHandler); - - setTimeout(() => { - pubSubClient - .subscription(subscriptionNameOrId, enableOpenTelemetryTracing) - .removeAllListeners(); - }, SUBSCRIBER_TIMEOUT * 1000); - } - - publishMessage().then(subscriptionListen()); - // [END opentelemetry_tracing] + const errorHandler = (error) => { + console.log('Received error:', error); + + console.log('Cleaning up OpenTelemetry exporter...'); + exporter.shutdown().then(() => { + // Cleaned up exporter. + process.exit(0); + }); + }; + + // Listens for new messages from the topic + pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); + pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); + + // Wait a bit for the subscription. For the sample only. + setTimeout(() => { + pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); + }, SUBSCRIBER_TIMEOUT * 1000); +} +// [END opentelemetry_tracing] + +function main( +topicNameOrId = 'YOUR_TOPIC_NAME_OR_ID', +subscriptionNameOrId = 'YOUR_SUBSCRIPTION_NAME_OR_ID', +data = 'Hello, world!') +{ + publishMessage(topicNameOrId, data). + then(() => subscriptionListen(subscriptionNameOrId)). + catch((err) => { + console.error(err.message); + process.exitCode = 1; + }); } -main(...process.argv.slice(2)); +main(...process.argv.slice(2)); \ No newline at end of file diff --git a/samples/package.json b/samples/package.json index 4c6c4e2ce..8d6b12b35 100644 --- a/samples/package.json +++ b/samples/package.json @@ -22,10 +22,10 @@ }, "dependencies": { "@google-cloud/pubsub": "^3.2.1", - "@opentelemetry/api": "^1.3.0", + "@opentelemetry/api": "~1.1.0", "@opentelemetry/resources": "~1.3.1", - "@opentelemetry/sdk-node": "~1.3.1", "@opentelemetry/sdk-trace-node": "~1.3.1", + "@opentelemetry/sdk-trace-base": "~1.3.1", "@opentelemetry/semantic-conventions": "~1.3.1", "avro-js": "^1.10.1", "p-defer": "^3.0.0", diff --git a/samples/typescript/openTelemetryTracing.ts b/samples/typescript/openTelemetryTracing.ts new file mode 100644 index 000000000..fc0b50fd2 --- /dev/null +++ b/samples/typescript/openTelemetryTracing.ts @@ -0,0 +1,128 @@ +// Copyright 2020-2022 Google LLC +// +// 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 +// +// 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. + +/** + * This sample demonstrates how to add OpenTelemetry tracing to the + * Google Cloud Pub/Sub API. + * + * For more information, see the README.md under /pubsub and the documentation + * at https://cloud.google.com/pubsub/docs. + */ + +// sample-metadata: +// title: OpenTelemetry Tracing +// description: Demonstrates how to enable OpenTelemetry tracing in +// a publisher or subscriber. +// usage: node openTelemetryTracing.js + +const SUBSCRIBER_TIMEOUT = 10; + +// [START opentelemetry_tracing] +/** + * TODO(developer): Uncomment these variables before running the sample. + */ +// const topicNameOrId = 'YOUR_TOPIC_OR_ID'; +// const subscriptionNameOrId = 'YOUR_SUBSCRIPTION_OR_ID'; +// const data = 'Hello, world!"; + +// Imports the Google Cloud client library +import {Message, PubSub} from '@google-cloud/pubsub'; + +// Imports the OpenTelemetry API +import * as otel from '@opentelemetry/sdk-trace-node'; +import {diag, DiagConsoleLogger, DiagLogLevel} from '@opentelemetry/api'; +const {NodeTracerProvider} = otel; +import { + SimpleSpanProcessor, + ConsoleSpanExporter, +} from '@opentelemetry/sdk-trace-base'; + +import {Resource} from '@opentelemetry/resources'; +import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; + +// Enable the diagnostic logger for OpenTelemetry +diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); + +const exporter = new ConsoleSpanExporter(); + +const provider = new NodeTracerProvider({ + resource: new Resource({ + [SemanticResourceAttributes.SERVICE_NAME]: 'otel example', + }), +}); +const processor = new SimpleSpanProcessor(exporter); +provider.addSpanProcessor(processor); +provider.register(); + +// Creates a client; cache this for further use. +const pubSubClient = new PubSub(); + +async function publishMessage(topicNameOrId: string, data: string) { + // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) + const dataBuffer = Buffer.from(data); + const messageId = await pubSubClient + .topic(topicNameOrId) + .publishMessage({data: dataBuffer}); + console.log(`Message ${messageId} published.`); +} + +async function subscriptionListen(subscriptionNameOrId: string) { + // Message handler for subscriber + const messageHandler = (message: Message) => { + console.log(`Message ${message.id} received.`); + message.ack(); + + // Ensure that all spans got flushed by the exporter + console.log('Cleaning up OpenTelemetry exporter...'); + exporter.shutdown().then(() => { + // Cleaned up exporter. + process.exit(0); + }); + }; + + const errorHandler = (error: Error) => { + console.log('Received error:', error); + + console.log('Cleaning up OpenTelemetry exporter...'); + exporter.shutdown().then(() => { + // Cleaned up exporter. + process.exit(0); + }); + }; + + // Listens for new messages from the topic + pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); + pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); + + // Wait a bit for the subscription. For the sample only. + setTimeout(() => { + pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); + }, SUBSCRIBER_TIMEOUT * 1000); +} +// [END opentelemetry_tracing] + +function main( + topicNameOrId = 'YOUR_TOPIC_NAME_OR_ID', + subscriptionNameOrId = 'YOUR_SUBSCRIPTION_NAME_OR_ID', + data = 'Hello, world!' +) { + publishMessage(topicNameOrId, data) + .then(() => subscriptionListen(subscriptionNameOrId)) + .catch(err => { + console.error(err.message); + process.exitCode = 1; + }); +} + +main(...process.argv.slice(2)); From 4a2a79e3ad75aa012e8f079d41ff3585f8a7cea9 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 16 Dec 2022 16:18:57 -0500 Subject: [PATCH 17/52] chore: linter nit --- src/publisher/index.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/publisher/index.ts b/src/publisher/index.ts index a83d57019..2ce471dd9 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -212,7 +212,7 @@ export class Publisher { } } - const span: Span | undefined = this.getParentSpan(message); + this.getParentSpan(message); if (!message.orderingKey) { this.queue.add(message, callback!); From bb815d3588be978f8fff784e69e2d62f7e35ee00 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 16 Dec 2022 16:21:22 -0500 Subject: [PATCH 18/52] chore: another linter nit --- test/lease-manager.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/lease-manager.ts b/test/lease-manager.ts index 6df782a63..dcc14e8bb 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -41,7 +41,7 @@ class FakeSubscriberTelemetry { flowEnd() {} schedulerStart() {} schedulerEnd() {} - processingStart(subName: string) {} + processingStart() {} processingEnd() {} } From 5d60ff3464599a92c462aa2dbba498b93a54a036 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Fri, 16 Dec 2022 16:23:22 -0500 Subject: [PATCH 19/52] chore: OTel sample formatting --- samples/openTelemetryTracing.js | 60 +++++++++++++++++---------------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index 4e5e8a088..557a860d3 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -41,19 +41,21 @@ const SUBSCRIBER_TIMEOUT = 10; // const data = 'Hello, world!"; // Imports the Google Cloud client library -const { PubSub } = require("@google-cloud/pubsub"); +const {PubSub} = require('@google-cloud/pubsub'); // Imports the OpenTelemetry API -const otel = require("@opentelemetry/sdk-trace-node"); -const { diag, DiagConsoleLogger, DiagLogLevel } = require("@opentelemetry/api"); -const { NodeTracerProvider } = otel; -const { SimpleSpanProcessor, ConsoleSpanExporter } = require("@opentelemetry/sdk-trace-base"); - - - - -const { Resource } = require("@opentelemetry/resources"); -const { SemanticResourceAttributes } = require("@opentelemetry/semantic-conventions"); +const otel = require('@opentelemetry/sdk-trace-node'); +const {diag, DiagConsoleLogger, DiagLogLevel} = require('@opentelemetry/api'); +const {NodeTracerProvider} = otel; +const { + SimpleSpanProcessor, + ConsoleSpanExporter, +} = require('@opentelemetry/sdk-trace-base'); + +const {Resource} = require('@opentelemetry/resources'); +const { + SemanticResourceAttributes, +} = require('@opentelemetry/semantic-conventions'); // Enable the diagnostic logger for OpenTelemetry diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); @@ -62,8 +64,8 @@ const exporter = new ConsoleSpanExporter(); const provider = new NodeTracerProvider({ resource: new Resource({ - [SemanticResourceAttributes.SERVICE_NAME]: 'otel example' - }) + [SemanticResourceAttributes.SERVICE_NAME]: 'otel example', + }), }); const processor = new SimpleSpanProcessor(exporter); provider.addSpanProcessor(processor); @@ -75,15 +77,15 @@ const pubSubClient = new PubSub(); async function publishMessage(topicNameOrId, data) { // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) const dataBuffer = Buffer.from(data); - const messageId = await pubSubClient. - topic(topicNameOrId). - publishMessage({ data: dataBuffer }); + const messageId = await pubSubClient + .topic(topicNameOrId) + .publishMessage({data: dataBuffer}); console.log(`Message ${messageId} published.`); } async function subscriptionListen(subscriptionNameOrId) { // Message handler for subscriber - const messageHandler = (message) => { + const messageHandler = message => { console.log(`Message ${message.id} received.`); message.ack(); @@ -95,7 +97,7 @@ async function subscriptionListen(subscriptionNameOrId) { }); }; - const errorHandler = (error) => { + const errorHandler = error => { console.log('Received error:', error); console.log('Cleaning up OpenTelemetry exporter...'); @@ -117,16 +119,16 @@ async function subscriptionListen(subscriptionNameOrId) { // [END opentelemetry_tracing] function main( -topicNameOrId = 'YOUR_TOPIC_NAME_OR_ID', -subscriptionNameOrId = 'YOUR_SUBSCRIPTION_NAME_OR_ID', -data = 'Hello, world!') -{ - publishMessage(topicNameOrId, data). - then(() => subscriptionListen(subscriptionNameOrId)). - catch((err) => { - console.error(err.message); - process.exitCode = 1; - }); + topicNameOrId = 'YOUR_TOPIC_NAME_OR_ID', + subscriptionNameOrId = 'YOUR_SUBSCRIPTION_NAME_OR_ID', + data = 'Hello, world!' +) { + publishMessage(topicNameOrId, data) + .then(() => subscriptionListen(subscriptionNameOrId)) + .catch(err => { + console.error(err.message); + process.exitCode = 1; + }); } -main(...process.argv.slice(2)); \ No newline at end of file +main(...process.argv.slice(2)); From f8a5c09e0c125610ca726fade1215a6ec3be671d Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Tue, 20 Dec 2022 16:02:40 -0500 Subject: [PATCH 20/52] chore: some internal renaming to reflect the function rather than name of telemetry tracing --- src/index.ts | 5 ++- src/publisher/flow-publisher.ts | 4 +-- src/publisher/index.ts | 14 ++++---- src/publisher/message-queues.ts | 6 ++-- src/publisher/pubsub-message.ts | 8 ++--- src/subscriber.ts | 34 ++++++++++--------- src/subscription.ts | 4 +-- ...emetry-tracing.ts => telemetry-tracing.ts} | 25 ++++++++++++++ test/publisher/index.ts | 2 +- ...emetry-tracing.ts => telemetry-tracing.ts} | 2 +- 10 files changed, 68 insertions(+), 36 deletions(-) rename src/{opentelemetry-tracing.ts => telemetry-tracing.ts} (95%) rename test/{opentelemetry-tracing.ts => telemetry-tracing.ts} (98%) diff --git a/src/index.ts b/src/index.ts index cb3f97a19..43b0bbcf3 100644 --- a/src/index.ts +++ b/src/index.ts @@ -184,4 +184,7 @@ if (process.env.DEBUG_GRPC) { import * as protos from '../protos/protos'; export {protos}; -export * as openTelemetry from './opentelemetry-tracing'; +// Deprecated; please see the updated OpenTelemetry sample +// for an example of how to use telemetry in this library. +import {legacyExports} from './telemetry-tracing'; +export {legacyExports as openTelemetry}; diff --git a/src/publisher/flow-publisher.ts b/src/publisher/flow-publisher.ts index adfffcb05..db18b0432 100644 --- a/src/publisher/flow-publisher.ts +++ b/src/publisher/flow-publisher.ts @@ -17,7 +17,7 @@ import {Publisher} from '.'; import {FlowControl} from './flow-control'; import {PubsubMessage, calculateMessageSize} from './pubsub-message'; -import * as otel from '../opentelemetry-tracing'; +import * as tracing from '../telemetry-tracing'; /** * Encapsulates a series of message publishes from a rapid loop (or similar @@ -78,7 +78,7 @@ export class FlowControlledPublisher { */ publish(message: PubsubMessage): Promise | null { const flowSpan = message.telemetrySpan - ? otel.SpanMaker.createPublishFlowSpan(message) + ? tracing.SpanMaker.createPublishFlowSpan(message) : undefined; const doPublish = () => { flowSpan?.end(); diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 2ce471dd9..d26ea6237 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -24,7 +24,7 @@ import {Queue, OrderedQueue} from './message-queues'; import {Topic} from '../topic'; import {RequestCallback, EmptyCallback} from '../pubsub'; import {defaultOptions} from '../default-options'; -import * as otel from '../opentelemetry-tracing'; +import * as tracing from '../telemetry-tracing'; import {FlowControl, FlowControlOptions} from './flow-control'; import {promisifySome} from '../util'; @@ -327,15 +327,14 @@ export class Publisher { } /** - * Finds or constructs an OpenTelemetry publish/parent span for a message, - * if OTel is enabled. + * Finds or constructs an telemetry publish/parent span for a message. * * @private * * @param {PubsubMessage} message The message to create a span for */ getParentSpan(message: PubsubMessage): Span | undefined { - const enabled = otel.isEnabled(this.settings); + const enabled = tracing.isEnabled(this.settings); if (!enabled) { return undefined; } @@ -344,11 +343,14 @@ export class Publisher { return message.telemetrySpan; } - const span = otel.SpanMaker.createPublisherSpan(message, this.topic.name); + const span = tracing.SpanMaker.createPublisherSpan( + message, + this.topic.name + ); // If the span's context is valid we should inject the propagation trace context. if (isSpanContextValid(span.spanContext())) { - otel.injectSpan(span, message, enabled); + tracing.injectSpan(span, message, enabled); } return span; diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 552471552..2e8ebcb5f 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -21,7 +21,7 @@ import {BatchPublishOptions, MessageBatch} from './message-batch'; import {PublishError} from './publish-error'; import {Publisher, PubsubMessage, PublishCallback} from './'; import {google} from '../../protos/protos'; -import * as otel from '../opentelemetry-tracing'; +import * as tracing from '../telemetry-tracing'; import {filterMessage} from './pubsub-message'; export interface PublishDone { @@ -99,7 +99,7 @@ export abstract class MessageQueue extends EventEmitter { } messages.forEach(m => { - const span = otel.SpanMaker.createPublishRpcSpan(m); + const span = tracing.SpanMaker.createPublishRpcSpan(m); if (span) { m.telemetryRpc = span; } @@ -161,7 +161,7 @@ export class Queue extends MessageQueue { this.publish(); } - message.telemetryBatching = otel.SpanMaker.createPublishBatchSpan(message); + message.telemetryBatching = tracing.SpanMaker.createPublishBatchSpan(message); this.batch.add(message, callback); diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index 47a443835..1d74642f3 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -15,7 +15,7 @@ */ import {google} from '../../protos/protos'; -import * as otel from '../opentelemetry-tracing'; +import * as tracing from '../telemetry-tracing'; /** * Strings are the only allowed values for keys and values in message attributes. @@ -27,7 +27,7 @@ export type Attributes = Record; */ export interface PubsubMessage extends google.pubsub.v1.IPubsubMessage, - otel.MessageWithAttributes { + tracing.MessageWithAttributes { /** * If we've calculated the size of this message, it will be cached here. * This is done to avoid having to build up the attribute size over and over. @@ -47,14 +47,14 @@ export interface PubsubMessage * * @private */ - telemetryBatching?: otel.Span; + telemetryBatching?: tracing.Span; /** * If telemetry is enabled, track the RPC send time span. * * @private */ - telemetryRpc?: otel.Span; + telemetryRpc?: tracing.Span; } /** diff --git a/src/subscriber.ts b/src/subscriber.ts index 4cd390982..29fdbe20a 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -26,7 +26,7 @@ import {MessageStream, MessageStreamOptions} from './message-stream'; import {Subscription} from './subscription'; import {defaultOptions} from './default-options'; import {SubscriberClient} from './v1'; -import * as otel from './opentelemetry-tracing'; +import * as tracing from './telemetry-tracing'; import {Duration} from './temporal'; import {EventEmitter} from 'events'; @@ -78,7 +78,7 @@ export class SubscriberTelemetry { flowStart() { if (!this.flow) { - this.flow = otel.SpanMaker.createReceiveFlowSpan(this.parent); + this.flow = tracing.SpanMaker.createReceiveFlowSpan(this.parent); } } @@ -91,7 +91,9 @@ export class SubscriberTelemetry { schedulerStart() { if (!this.scheduler) { - this.scheduler = otel.SpanMaker.createReceiveSchedulerSpan(this.parent); + this.scheduler = tracing.SpanMaker.createReceiveSchedulerSpan( + this.parent + ); } } @@ -104,7 +106,7 @@ export class SubscriberTelemetry { processingStart(subName: string) { if (!this.processing) { - this.processing = otel.SpanMaker.createReceiveProcessSpan( + this.processing = tracing.SpanMaker.createReceiveProcessSpan( this.parent, subName ); @@ -118,9 +120,9 @@ export class SubscriberTelemetry { } } - private flow?: otel.Span; - private scheduler?: otel.Span; - private processing?: otel.Span; + private flow?: tracing.Span; + private scheduler?: tracing.Span; + private processing?: tracing.Span; } /** @@ -150,7 +152,7 @@ export class SubscriberTelemetry { * }); * ``` */ -export class Message implements otel.MessageWithAttributes { +export class Message implements tracing.MessageWithAttributes { ackId: string; attributes: {[key: string]: string}; data: Buffer; @@ -173,7 +175,7 @@ export class Message implements otel.MessageWithAttributes { * implement a private interface seems to confuse TypeScript. (And it's needed * in unit tests.) */ - telemetrySpan?: otel.Span; + telemetrySpan?: tracing.Span; /** * @private @@ -614,7 +616,7 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, true); + const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, true); // Ignore this in this version of the method (but hook then/catch // to avoid unhandled exceptions). @@ -641,7 +643,7 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, true); + const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, true); await this._acks.add(message); @@ -746,7 +748,7 @@ export class Subscriber extends EventEmitter { * @private */ async nack(message: Message): Promise { - const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, false); + const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, false); await this.modAck(message, 0); @@ -765,7 +767,7 @@ export class Subscriber extends EventEmitter { * @private */ async nackWithResponse(message: Message): Promise { - const ackSpan = otel.SpanMaker.createReceiveResponseSpan(message, false); + const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, false); const response = await this.modAckWithResponse(message, 0); ackSpan?.end(); return response; @@ -843,17 +845,17 @@ export class Subscriber extends EventEmitter { } /** - * Constructs an OpenTelemetry span from the incoming message. + * Constructs a telemetry span from the incoming message. * * @param {Message} message One of the received messages * @private */ private createParentSpan(message: Message): void { - const enabled = otel.isEnabled({ + const enabled = tracing.isEnabled({ enableOpenTelemetryTracing: this._useLegacyOpenTelemetry, }); if (enabled) { - otel.extractSpan(message, this.name, enabled); + tracing.extractSpan(message, this.name, enabled); } } diff --git a/src/subscription.ts b/src/subscription.ts index 031c1a175..0e820caff 100644 --- a/src/subscription.ts +++ b/src/subscription.ts @@ -48,7 +48,7 @@ import {promisifySome} from './util'; export {AckError, AckResponse, AckResponses} from './subscriber'; import {EmitterCallback, WrappingEmitter} from './wrapping-emitter'; -import * as otel from './opentelemetry-tracing'; +import * as tracing from './telemetry-tracing'; export type PushConfig = google.pubsub.v1.IPushConfig; export type OidcToken = google.pubsub.v1.PushConfig.IOidcToken; @@ -353,7 +353,7 @@ export class Subscription extends WrappingEmitter { if (eventName !== 'message') { return listener(...args); } else { - const span = otel.SpanMaker.createReceiveProcessSpan( + const span = tracing.SpanMaker.createReceiveProcessSpan( args[0] as Message, this.name ); diff --git a/src/opentelemetry-tracing.ts b/src/telemetry-tracing.ts similarity index 95% rename from src/opentelemetry-tracing.ts rename to src/telemetry-tracing.ts index 18fdb6f93..ca8c9bdd4 100644 --- a/src/opentelemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -441,3 +441,28 @@ export function extractSpan( message.telemetrySpan = span; return span; } + +// Since these were exported on the main Pub/Sub index in the previous +// version, we have to export them until the next major. +export const legacyExports = { + /** + * @deprecated + * Use the new telemetry functionality instead; see the updated OpenTelemetry + * sample for an example. + */ + createSpan: function ( + spanName: string, + kind: SpanKind, + attributes?: SpanAttributes, + parent?: SpanContext + ): Span { + return getTracer().startSpan( + spanName, + { + kind, + attributes, + }, + parent ? trace.setSpanContext(context.active(), parent) : undefined + ); + }, +}; diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 6e3453252..6203fe3ea 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -386,7 +386,7 @@ describe('Publisher', () => { it('should issue a warning if OpenTelemetry span context key is set', () => { const warnSpy = sinon.spy(console, 'warn'); const attributes = { - [otel.legacyAttributeName]: 'foobar', + [tracing.legacyAttributeName]: 'foobar', }; const fakeMessageWithOTKey = {data, attributes}; const publisherTracing = new Publisher(topic, { diff --git a/test/opentelemetry-tracing.ts b/test/telemetry-tracing.ts similarity index 98% rename from test/opentelemetry-tracing.ts rename to test/telemetry-tracing.ts index bafd748d0..8ce9a203d 100644 --- a/test/opentelemetry-tracing.ts +++ b/test/telemetry-tracing.ts @@ -18,7 +18,7 @@ import * as assert from 'assert'; import {describe, it, beforeEach} from 'mocha'; import * as trace from '@opentelemetry/sdk-trace-base'; -import * as otel from '../src/opentelemetry-tracing'; +import * as otel from '../src/telemetry-tracing'; import {exporter} from './tracing'; import {SpanKind} from '@opentelemetry/api'; import sinon = require('sinon'); From d21480dc6061b755c04d8c6059981a4d1bd70b63 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 21 Dec 2022 16:32:00 -0500 Subject: [PATCH 21/52] fix: more work towards the Promise-first publish flow --- src/publisher/index.ts | 5 +- src/publisher/message-queues.ts | 146 ++++++++++++++++++------------- test/publisher/index.ts | 32 +++---- test/publisher/message-queues.ts | 97 ++++++++++---------- 4 files changed, 144 insertions(+), 136 deletions(-) diff --git a/src/publisher/index.ts b/src/publisher/index.ts index d26ea6237..397192bac 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -14,7 +14,6 @@ * limitations under the License. */ -import {promisify} from '@google-cloud/promisify'; import * as extend from 'extend'; import {CallOptions} from 'google-gax'; import {isSpanContextValid, Span} from '@opentelemetry/api'; @@ -129,9 +128,7 @@ export class Publisher { ) ); - const allPublishes = Promise.all( - toDrain.map(q => promisify(q.publish).bind(q)()) - ); + const allPublishes = Promise.all(toDrain.map(q => q.publish.bind(q)())); allPublishes .then(() => allDrains) diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 2e8ebcb5f..f38f1a8ce 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -23,10 +23,7 @@ import {Publisher, PubsubMessage, PublishCallback} from './'; import {google} from '../../protos/protos'; import * as tracing from '../telemetry-tracing'; import {filterMessage} from './pubsub-message'; - -export interface PublishDone { - (err: ServiceError | null): void; -} +import {promisify} from 'util'; /** * Queues are used to manage publishing batches of messages. @@ -73,31 +70,45 @@ export abstract class MessageQueue extends EventEmitter { * * @abstract */ - abstract publish(): void; + abstract publish(): Promise; /** * Accepts a batch of messages and publishes them to the API. * * @param {object[]} messages The messages to publish. * @param {PublishCallback[]} callbacks The corresponding callback functions. - * @param {function} [callback] Callback to be fired when publish is done. */ - _publish( + async _publish( messages: PubsubMessage[], - callbacks: PublishCallback[], - callback?: PublishDone - ): void { + callbacks: PublishCallback[] + ): Promise { const {topic, settings} = this.publisher; const reqOpts = { topic: topic.name, messages: messages.map(filterMessage), }; if (messages.length === 0) { - if (typeof callback === 'function') { - callback(null); - } return; } + // Make sure we have a projectId filled in to update telemetry spans. + // The overall spans may not have the correct projectId because it wasn't + // known at the time publishMessage was called. + const anySpans = !!messages.find(m => m.telemetrySpan); + if (anySpans) { + if (!topic.pubsub.isIdResolved) { + await topic.pubsub.getClientConfig(); + } + + messages.forEach(m => { + if (m.telemetrySpan) { + tracing.SpanMaker.updatePublisherTopicName( + m.telemetrySpan, + topic.name + ); + } + }); + } + messages.forEach(m => { const span = tracing.SpanMaker.createPublishRpcSpan(m); if (span) { @@ -105,27 +116,31 @@ export abstract class MessageQueue extends EventEmitter { } }); - topic.request( - { + const requestCallback = topic.request; + const request = promisify(requestCallback.bind(topic)); + try { + const resp = await request({ client: 'PublisherClient', method: 'publish', reqOpts, gaxOpts: settings.gaxOpts!, - }, - (err, resp) => { - messages.forEach(m => { - m.telemetryRpc?.end(); - m.telemetrySpan?.end(); - }); - - const messageIds = (resp && resp.messageIds) || []; - callbacks.forEach((callback, i) => callback(err, messageIds[i])); - - if (typeof callback === 'function') { - callback(err); - } + }); + + if (resp) { + const messageIds = resp.messageIds || []; + callbacks.forEach((callback, i) => callback(null, messageIds[i])); } - ); + } catch (e) { + const err = e as ServiceError; + callbacks.forEach(callback => callback(err)); + + throw e; + } finally { + messages.forEach(m => { + m.telemetryRpc?.end(); + m.telemetrySpan?.end(); + }); + } } } @@ -158,18 +173,24 @@ export class Queue extends MessageQueue { */ add(message: PubsubMessage, callback: PublishCallback): void { if (!this.batch.canFit(message)) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } - message.telemetryBatching = tracing.SpanMaker.createPublishBatchSpan(message); + message.telemetryBatching = + tracing.SpanMaker.createPublishBatchSpan(message); this.batch.add(message, callback); if (this.batch.isFull()) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } else if (!this.pending) { const {maxMilliseconds} = this.batchOptions; - this.pending = setTimeout(() => this.publish(), maxMilliseconds!); + this.pending = setTimeout(() => { + // Ignore errors. + this.publish().catch(() => {}); + }, maxMilliseconds!); } } /** @@ -177,8 +198,7 @@ export class Queue extends MessageQueue { * * @emits Queue#drain when all messages are sent. */ - publish(callback?: PublishDone): void { - const definedCallback = callback || (() => {}); + async publish(): Promise { const {messages, callbacks} = this.batch; this.batch = new MessageBatch(this.batchOptions); @@ -190,17 +210,13 @@ export class Queue extends MessageQueue { messages.forEach(m => m.telemetryBatching?.end()); - this._publish(messages, callbacks, (err: null | ServiceError) => { - if (err) { - definedCallback(err); - } else if (this.batch.messages.length) { - // Make another go-around, we're trying to drain the queues fully. - this.publish(callback); - } else { - this.emit('drain'); - definedCallback(null); - } - }); + await this._publish(messages, callbacks); + if (this.batch.messages.length) { + // Make another go-around, we're trying to drain the queues fully. + await this.publish(); + } else { + this.emit('drain'); + } } } @@ -267,7 +283,8 @@ export class OrderedQueue extends MessageQueue { } if (!this.currentBatch.canFit(message)) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } this.currentBatch.add(message, callback); @@ -276,7 +293,8 @@ export class OrderedQueue extends MessageQueue { // check again here if (!this.inFlight) { if (this.currentBatch.isFull()) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } else if (!this.pending) { this.beginNextPublish(); } @@ -290,7 +308,10 @@ export class OrderedQueue extends MessageQueue { const timeWaiting = Date.now() - this.currentBatch.created; const delay = Math.max(0, maxMilliseconds - timeWaiting); - this.pending = setTimeout(() => this.publish(), delay); + this.pending = setTimeout(() => { + // Ignore errors. + this.publish().catch(() => {}); + }, delay); } /** * Creates a new {@link MessageBatch} instance. @@ -325,8 +346,7 @@ export class OrderedQueue extends MessageQueue { * * @fires OrderedQueue#drain */ - publish(callback?: PublishDone): void { - const definedCallback = callback || (() => {}); + async publish(): Promise { this.inFlight = true; if (this.pending) { @@ -336,19 +356,21 @@ export class OrderedQueue extends MessageQueue { const {messages, callbacks} = this.batches.pop()!; - this._publish(messages, callbacks, (err: null | ServiceError) => { + try { + await this._publish(messages, callbacks); + } catch (e) { + const err = e as ServiceError; + this.inFlight = false; + this.handlePublishFailure(err); + } finally { this.inFlight = false; + } - if (err) { - this.handlePublishFailure(err); - definedCallback(err); - } else if (this.batches.length) { - this.beginNextPublish(); - } else { - this.emit('drain'); - definedCallback(null); - } - }); + if (this.batches.length) { + this.beginNextPublish(); + } else { + this.emit('drain'); + } } /** diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 6203fe3ea..cfbfb76d8 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -28,7 +28,7 @@ import {PublishError} from '../../src/publisher/publish-error'; import * as util from '../../src/util'; import {defaultOptions} from '../../src/default-options'; -import * as otel from '../../src/opentelemetry-tracing'; +import * as tracing from '../../src/telemetry-tracing'; import {exporter} from '../tracing'; import {SpanKind} from '@opentelemetry/api'; import {SemanticAttributes} from '@opentelemetry/semantic-conventions'; @@ -61,16 +61,14 @@ class FakeQueue extends EventEmitter { updateOptions() {} // eslint-disable-next-line @typescript-eslint/no-unused-vars add(message: p.PubsubMessage, callback: p.PublishCallback): void {} - publish(callback: (err: Error | null) => void) { - this._publish([], [], callback); + async publish() { + await this._publish([], []); } - _publish( + async _publish( // eslint-disable-next-line @typescript-eslint/no-unused-vars messages: p.PubsubMessage[], // eslint-disable-next-line @typescript-eslint/no-unused-vars - callbacks: p.PublishCallback[], - // eslint-disable-next-line @typescript-eslint/no-unused-vars - callback?: q.PublishDone + callbacks: p.PublishCallback[] ) {} } @@ -82,17 +80,14 @@ class FakeOrderedQueue extends FakeQueue { this.orderingKey = key; } resumePublishing(): void {} - // eslint-disable-next-line @typescript-eslint/no-unused-vars - publish(callback: (err: Error | null) => void) { - this._publish([], [], callback); + async publish() { + await this._publish([], []); } - _publish( + async _publish( // eslint-disable-next-line @typescript-eslint/no-unused-vars messages: p.PubsubMessage[], // eslint-disable-next-line @typescript-eslint/no-unused-vars - callbacks: p.PublishCallback[], - // eslint-disable-next-line @typescript-eslint/no-unused-vars - callback?: q.PublishDone + callbacks: p.PublishCallback[] ) {} } @@ -349,18 +344,17 @@ describe('Publisher', () => { // We have to stub out the regular queue as well, so that the flush() operation finishes. sandbox .stub(FakeQueue.prototype, '_publish') - .callsFake((messages, callbacks, callback) => { + .callsFake(async (messages, callbacks) => { // Simulate the drain taking longer than the publishes. This can // happen if more messages are queued during the publish(). process.nextTick(() => { publisher.queue.emit('drain'); }); - if (typeof callback === 'function') callback(null); }); sandbox .stub(FakeOrderedQueue.prototype, '_publish') - .callsFake((messages, callbacks, callback) => { + .callsFake(async (messages, callbacks) => { const queue = publisher.orderedQueues.get( orderingKey ) as unknown as FakeOrderedQueue; @@ -370,7 +364,6 @@ describe('Publisher', () => { process.nextTick(() => { queue.emit('drain'); }); - if (typeof callback === 'function') callback(null); }); publisher.orderedQueues.clear(); @@ -507,13 +500,12 @@ describe('Publisher', () => { it('should drain the main publish queue', done => { sandbox .stub(publisher.queue, '_publish') - .callsFake((messages, callbacks, callback) => { + .callsFake(async (messages, callbacks) => { // Simulate the drain taking longer than the publishes. This can // happen if more messages are queued during the publish(). process.nextTick(() => { publisher.queue.emit('drain'); }); - if (typeof callback === 'function') callback(null); }); publisher.flush(err => { diff --git a/test/publisher/message-queues.ts b/test/publisher/message-queues.ts index 7ea076477..0e48a7cfa 100644 --- a/test/publisher/message-queues.ts +++ b/test/publisher/message-queues.ts @@ -167,42 +167,41 @@ describe('Message Queues', () => { assert.strictEqual(gaxOpts, callOptions); }); - it('should pass back any request errors', done => { + it('should pass back any request errors', async () => { const error = new Error('err') as ServiceError; sandbox.stub(topic, 'request').callsFake((config, callback) => { callback(error); }); - queue._publish(messages, callbacks, err => { + try { + await queue._publish(messages, callbacks); + assert.strictEqual(null, error, '_publish did not throw'); + } catch (e) { + const err = e as ServiceError; + assert.strictEqual(err, error); callbacks.forEach(callback => { const [err] = callback.lastCall.args; assert.strictEqual(err, error); }); - - done(); - }); + } }); - it('should pass back message ids', done => { + it('should pass back message ids', async () => { const messageIds = messages.map((_, i) => `message${i}`); sandbox.stub(topic, 'request').callsFake((config, callback) => { callback(null, {messageIds}); }); - queue._publish(messages, callbacks, err => { - assert.ifError(err); - - callbacks.forEach((callback, i) => { - const [, messageId] = callback.lastCall.args; - const expectedId = `message${i}`; - assert.strictEqual(messageId, expectedId); - }); + await queue._publish(messages, callbacks); - done(); + callbacks.forEach((callback, i) => { + const [, messageId] = callback.lastCall.args; + const expectedId = `message${i}`; + assert.strictEqual(messageId, expectedId); }); }); }); @@ -239,20 +238,19 @@ describe('Message Queues', () => { const addStub = sandbox.stub(queue.batch, 'add'); sandbox.stub(queue.batch, 'canFit').returns(false); - sandbox - .stub(queue, 'publish') - .onCall(0) - .callsFake(() => { - assert.strictEqual(addStub.callCount, 0); - done(); - }); + const publishStub = sandbox.stub(queue, 'publish'); + publishStub.onCall(0).callsFake(async () => { + assert.strictEqual(addStub.callCount, 0); + done(); + }); + publishStub.resolves(); queue.add(fakeMessage, spy); }); it('should add the message to the batch', () => { const stub = sandbox.stub(queue.batch, 'add'); - sandbox.stub(queue, 'publish'); + sandbox.stub(queue, 'publish').resolves(); queue.add(fakeMessage, spy); @@ -262,7 +260,7 @@ describe('Message Queues', () => { }); it('should publish immediately if the batch became full', () => { - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); sandbox.stub(queue.batch, 'isFull').returns(true); queue.add(fakeMessage, spy); @@ -272,7 +270,7 @@ describe('Message Queues', () => { it('should set a timeout to publish if need be', () => { const clock = sandbox.useFakeTimers(); - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); const maxMilliseconds = 1234; queue.batchOptions = {maxMilliseconds}; @@ -286,7 +284,7 @@ describe('Message Queues', () => { it('should noop if a timeout is already set', () => { const clock = sandbox.useFakeTimers(); - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); const maxMilliseconds = 1234; queue.batchOptions = {maxMilliseconds}; @@ -435,13 +433,12 @@ describe('Message Queues', () => { const addStub = sandbox.stub(batch, 'add'); sandbox.stub(batch, 'canFit').withArgs(fakeMessage).returns(false); - sandbox - .stub(queue, 'publish') - .onCall(0) - .callsFake(() => { - assert.strictEqual(addStub.callCount, 0); - done(); - }); + const publishStub = sandbox.stub(queue, 'publish'); + publishStub.onCall(0).callsFake(async () => { + assert.strictEqual(addStub.callCount, 0); + done(); + }); + publishStub.resolves(); queue.add(fakeMessage, spy); }); @@ -457,12 +454,12 @@ describe('Message Queues', () => { }); it('should noop after adding if a publish was triggered', () => { - const publishStub = sandbox.stub(queue, 'publish'); + const publishStub = sandbox.stub(queue, 'publish').resolves(); const beginPublishStub = sandbox.stub(queue, 'beginNextPublish'); sandbox.stub(batch, 'canFit').returns(false); - publishStub.onCall(0).callsFake(() => { + publishStub.onCall(0).callsFake(async () => { queue.inFlight = true; }); @@ -473,7 +470,7 @@ describe('Message Queues', () => { }); it('should publish immediately if the batch is full', () => { - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); sandbox.stub(batch, 'isFull').returns(true); queue.add(fakeMessage, spy); @@ -514,14 +511,14 @@ describe('Message Queues', () => { }); it('should set a timeout that will call publish', done => { - sandbox.stub(queue, 'publish').callsFake(done); + sandbox.stub(queue, 'publish').callsFake(async () => done()); queue.beginNextPublish(); clock.tick(maxMilliseconds); }); it('should factor in the time the batch has been sitting', done => { const halfway = maxMilliseconds / 2; - sandbox.stub(queue, 'publish').callsFake(done); + sandbox.stub(queue, 'publish').callsFake(async () => done()); queue.currentBatch.created = Date.now() - halfway; queue.beginNextPublish(); clock.tick(halfway); @@ -618,46 +615,46 @@ describe('Message Queues', () => { assert.strictEqual(callbacks, spies); }); - it('should set inFlight to false after publishing', () => { - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + it('should set inFlight to false after publishing', async () => { + sandbox.stub(queue, '_publish').resolves(); - queue.publish(); + await queue.publish(); assert.strictEqual(queue.inFlight, false); }); - it('should handle any publish failures', () => { + it('should handle any publish failures', async () => { const error = new Error('err') as ServiceError; const stub = sandbox.stub(queue, 'handlePublishFailure'); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(error)); + sandbox.stub(queue, '_publish').rejects(error); - queue.publish(); + await queue.publish(); const [err] = stub.lastCall.args; assert.strictEqual(err, error); }); - it('should begin another publish if there are pending batches', () => { + it('should begin another publish if there are pending batches', async () => { const stub = sandbox.stub(queue, 'beginNextPublish'); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').resolves(); const secondBatch = new FakeMessageBatch(); secondBatch.messages = fakeMessages; secondBatch.callbacks = spies; queue.batches.push(secondBatch as b.MessageBatch); - queue.publish(); + await queue.publish(); assert.strictEqual(stub.callCount, 1); }); - it('should emit "drain" if there is nothing left to publish', () => { + it('should emit "drain" if there is nothing left to publish', async () => { const spy = sandbox.spy(); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').resolves(); queue.on('drain', spy); - queue.publish(); + await queue.publish(); assert.strictEqual(spy.callCount, 1); }); From fe4947df856bdfc4e03f97e3be92314bded1d175 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 21 Dec 2022 16:33:41 -0500 Subject: [PATCH 22/52] tests: remove unnecessary init line --- test/pubsub.ts | 1 - 1 file changed, 1 deletion(-) diff --git a/test/pubsub.ts b/test/pubsub.ts index eef9fccdc..99acdc74a 100644 --- a/test/pubsub.ts +++ b/test/pubsub.ts @@ -187,7 +187,6 @@ describe('PubSub', () => { v1ClientOverrides = {}; googleAuthOverride = null; pubsub = new PubSub(OPTIONS); - pubsub.projectId = PROJECT_ID; }); describe('instantiation', () => { From 568add9b5fd7d2f73d4e13b9fd5dc90a6c29e4b1 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 21 Dec 2022 16:34:19 -0500 Subject: [PATCH 23/52] fix: add tracing support for updating the topic name in publisher spans --- src/telemetry-tracing.ts | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index ca8c9bdd4..e9088a210 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -229,6 +229,11 @@ export class SpanMaker { return span; } + static updatePublisherTopicName(span: Span, topicName: string) { + span.updateName(`${topicName} send`); + span.setAttribute(SemanticAttributes.MESSAGING_DESTINATION, topicName); + } + static createReceiveSpan( message: MessageWithAttributes, subName: string, From 5243ea9e4fc97542e86f8d4ff49604ffb541ebd1 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 21 Dec 2022 17:43:02 -0500 Subject: [PATCH 24/52] fix: use the most up to date name for topic, subscription, and iam --- src/iam.ts | 25 ++++++++++++++++++++++--- src/subscription.ts | 12 +++++++++--- src/topic.ts | 13 +++++++++---- test/subscription.ts | 2 +- test/topic.ts | 2 +- 5 files changed, 42 insertions(+), 12 deletions(-) diff --git a/src/iam.ts b/src/iam.ts index ea6128be3..db4da053e 100644 --- a/src/iam.ts +++ b/src/iam.ts @@ -34,6 +34,15 @@ export type SetPolicyCallback = RequestCallback; export type SetPolicyResponse = [Policy]; export type GetPolicyResponse = [Policy]; +/** + * Allows us to get the most up to date full name of an object. + * + * @private + */ +export interface Nameable { + name: string; +} + /** * Shows which IAM permissions is allowed. * The key to this object are the IAM permissions (string) and the values are @@ -95,12 +104,22 @@ export type TestIamPermissionsCallback = ResourceCallback< export class IAM { pubsub: PubSub; request: typeof PubSub.prototype.request; - id: string; + private nameable_: Nameable; - constructor(pubsub: PubSub, id: string) { + constructor(pubsub: PubSub, nameOrNameable: Nameable | string) { this.pubsub = pubsub; this.request = pubsub.request.bind(pubsub); - this.id = id; + if (typeof nameOrNameable === 'string') { + this.nameable_ = { + name: nameOrNameable, + }; + } else { + this.nameable_ = nameOrNameable; + } + } + + get id(): string { + return this.nameable_.name; } /** diff --git a/src/subscription.ts b/src/subscription.ts index 0e820caff..62f9ead96 100644 --- a/src/subscription.ts +++ b/src/subscription.ts @@ -271,11 +271,12 @@ export type DetachSubscriptionResponse = EmptyResponse; export class Subscription extends WrappingEmitter { pubsub: PubSub; iam: IAM; - name: string; topic?: Topic | string; metadata?: google.pubsub.v1.ISubscription; request: typeof PubSub.prototype.request; + private _subscriber: Subscriber; + constructor(pubsub: PubSub, name: string, options?: SubscriptionOptions) { super(); @@ -285,7 +286,7 @@ export class Subscription extends WrappingEmitter { this.pubsub = pubsub; this.request = pubsub.request.bind(pubsub); - this.name = Subscription.formatName_(this.projectId, name); + this.id_ = name; this.topic = options.topic; /** @@ -326,7 +327,7 @@ export class Subscription extends WrappingEmitter { * }); * ``` */ - this.iam = new IAM(pubsub, this.name); + this.iam = new IAM(pubsub, this); this._subscriber = new Subscriber(this, options); this._subscriber @@ -338,6 +339,11 @@ export class Subscription extends WrappingEmitter { this._listen(); } + private id_: string; + get name(): string { + return Subscription.formatName_(this.pubsub.projectId, this.id_); + } + /** * This wrapper will be called as part of the emit() process. This lets * us capture the full time span of processing even if the user is using diff --git a/src/topic.ts b/src/topic.ts index 03a54c37c..3fc91e632 100644 --- a/src/topic.ts +++ b/src/topic.ts @@ -105,7 +105,6 @@ export type MessageOptions = PubsubMessage & {json?: any}; * ``` */ export class Topic { - name: string; parent: PubSub; pubsub: PubSub; request: typeof PubSub.prototype.request; @@ -118,11 +117,12 @@ export class Topic { constructor(pubsub: PubSub, name: string, options?: PublishOptions) { /** - * The fully qualified name of this topic. + * The fully qualified name of this topic. May have a placeholder for + * the projectId if it's not been resolved. * @name Topic#name * @type {string} */ - this.name = Topic.formatName_(pubsub.projectId, name); + this.id_ = name; this.publisher = new Publisher(this, options); /** * The parent {@link PubSub} instance of this topic instance. @@ -179,7 +179,12 @@ export class Topic { * }); * ``` */ - this.iam = new IAM(pubsub, this.name); + this.iam = new IAM(pubsub, this); + } + + private id_: string; + get name(): string { + return Topic.formatName_(this.parent.projectId, this.id_); } /** diff --git a/test/subscription.ts b/test/subscription.ts index 1fdaa36f9..051ebc95f 100644 --- a/test/subscription.ts +++ b/test/subscription.ts @@ -168,7 +168,7 @@ describe('Subscription', () => { assert(subscription.iam instanceof FakeIAM); const args = (subscription.iam as {} as FakeIAM).calledWith_; assert.strictEqual(args[0], PUBSUB); - assert.strictEqual(args[1], subscription.name); + assert.strictEqual(args[1], subscription); }); it('should create a Subscriber', () => { diff --git a/test/topic.ts b/test/topic.ts index bbd0aa2df..d2a103a06 100644 --- a/test/topic.ts +++ b/test/topic.ts @@ -188,7 +188,7 @@ describe('Topic', () => { }); it('should create an iam object', () => { - assert.deepStrictEqual(topic.iam.calledWith_, [PUBSUB, TOPIC_NAME]); + assert.deepStrictEqual(topic.iam.calledWith_, [PUBSUB, topic]); }); }); From 8dd13f804858dc5a867afff922a2bfc5d92d3803 Mon Sep 17 00:00:00 2001 From: Megan Potter Date: Wed, 21 Dec 2022 17:49:45 -0500 Subject: [PATCH 25/52] fix: update telemetry spans before publishing, in case the topic name wasn't ready --- src/publisher/message-queues.ts | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index f38f1a8ce..624440c37 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -93,19 +93,16 @@ export abstract class MessageQueue extends EventEmitter { // Make sure we have a projectId filled in to update telemetry spans. // The overall spans may not have the correct projectId because it wasn't // known at the time publishMessage was called. - const anySpans = !!messages.find(m => m.telemetrySpan); - if (anySpans) { + const spanMessages = messages.filter(m => !!m.telemetrySpan); + if (spanMessages.length) { if (!topic.pubsub.isIdResolved) { await topic.pubsub.getClientConfig(); } - - messages.forEach(m => { - if (m.telemetrySpan) { - tracing.SpanMaker.updatePublisherTopicName( - m.telemetrySpan, - topic.name - ); - } + spanMessages.forEach(m => { + tracing.SpanMaker.updatePublisherTopicName( + m.telemetrySpan!, + topic.name + ); }); } From bc9503741e007032d64d2c4c81ce588f96afab08 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 26 Apr 2023 15:19:11 -0400 Subject: [PATCH 26/52] chore: calm linter qualms --- samples/typescript/openTelemetryTracing.ts | 2 +- src/telemetry-tracing.ts | 2 +- src/wrapping-emitter.ts | 2 +- test/telemetry-tracing.ts | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/samples/typescript/openTelemetryTracing.ts b/samples/typescript/openTelemetryTracing.ts index fc0b50fd2..bb68af87a 100644 --- a/samples/typescript/openTelemetryTracing.ts +++ b/samples/typescript/openTelemetryTracing.ts @@ -1,4 +1,4 @@ -// Copyright 2020-2022 Google LLC +// Copyright 2020-2023 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index e9088a210..1d2e54f91 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -1,5 +1,5 @@ /*! - * Copyright 2020 Google LLC + * Copyright 2020-2023 Google LLC * 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 diff --git a/src/wrapping-emitter.ts b/src/wrapping-emitter.ts index 2bbd8c370..de3fbfe60 100644 --- a/src/wrapping-emitter.ts +++ b/src/wrapping-emitter.ts @@ -1,4 +1,4 @@ -// Copyright 2022 Google LLC +// Copyright 2022-2023 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/telemetry-tracing.ts b/test/telemetry-tracing.ts index 8ce9a203d..de450d580 100644 --- a/test/telemetry-tracing.ts +++ b/test/telemetry-tracing.ts @@ -1,5 +1,5 @@ /*! - * Copyright 2020 Google LLC + * Copyright 2020-2023 Google LLC * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From a3d66c298e60a5ff47b95ebfa6799efbbd510dbb Mon Sep 17 00:00:00 2001 From: Owl Bot Date: Wed, 26 Apr 2023 19:23:44 +0000 Subject: [PATCH 27/52] =?UTF-8?q?=F0=9F=A6=89=20Updates=20from=20OwlBot=20?= =?UTF-8?q?post-processor?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md --- samples/openTelemetryTracing.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index 557a860d3..bed6c9bd2 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -1,4 +1,4 @@ -// Copyright 2020-2022 Google LLC +// Copyright 2020-2023 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 90f9eadfe8941a9eae5a311f4a7deb3e1d16aa0d Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Fri, 28 Apr 2023 16:33:54 -0400 Subject: [PATCH 28/52] fix: update drain() not to promisify --- src/publisher/index.ts | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 45f2845cf..47429255d 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -124,14 +124,12 @@ export class Publisher { // event listeners after we've completed flush(). q.removeListener('drain', flushResolver); }; - return q.on('drain', flushResolver); + q.on('drain', flushResolver); }) ) ); - const allPublishes = Promise.all( - toDrain.map(q => promisify(q.publishDrain).bind(q)()) - ); + const allPublishes = Promise.all(toDrain.map(q => q.publishDrain())); allPublishes .then(() => allDrains) From 4ce3fc4ff00fa3750979a688defe498720f61832 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Fri, 28 Apr 2023 16:34:27 -0400 Subject: [PATCH 29/52] tests: update unit tests for recent promise changes to publish() --- test/publisher/index.ts | 30 +++++++++++------------- test/publisher/message-queues.ts | 40 ++++++++++++++++---------------- 2 files changed, 33 insertions(+), 37 deletions(-) diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 4c5b1d2d4..c30373e90 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -348,19 +348,17 @@ describe('Publisher', () => { it('should drain any ordered queues on flush', done => { // We have to stub out the regular queue as well, so that the flush() operation finishes. - sandbox - .stub(FakeQueue.prototype, '_publish') - .callsFake(async (messages, callbacks) => { - // Simulate the drain taking longer than the publishes. This can - // happen if more messages are queued during the publish(). - process.nextTick(() => { - publisher.queue.emit('drain'); - }); + sandbox.stub(FakeQueue.prototype, '_publish').callsFake(async () => { + // Simulate the drain taking longer than the publishes. This can + // happen if more messages are queued during the publish(). + process.nextTick(() => { + publisher.queue.emit('drain'); }); + }); sandbox .stub(FakeOrderedQueue.prototype, '_publish') - .callsFake(async (messages, callbacks) => { + .callsFake(async () => { const queue = publisher.orderedQueues.get( orderingKey ) as unknown as FakeOrderedQueue; @@ -504,15 +502,13 @@ describe('Publisher', () => { describe('flush', () => { // The ordered queue drain test is above with the ordered queue tests. it('should drain the main publish queue', done => { - sandbox - .stub(publisher.queue, '_publish') - .callsFake(async (messages, callbacks) => { - // Simulate the drain taking longer than the publishes. This can - // happen if more messages are queued during the publish(). - process.nextTick(() => { - publisher.queue.emit('drain'); - }); + sandbox.stub(publisher.queue, '_publish').callsFake(async () => { + // Simulate the drain taking longer than the publishes. This can + // happen if more messages are queued during the publish(). + process.nextTick(() => { + publisher.queue.emit('drain'); }); + }); publisher.flush(err => { assert.strictEqual(err, null); diff --git a/test/publisher/message-queues.ts b/test/publisher/message-queues.ts index 3adc90842..f28b4e33b 100644 --- a/test/publisher/message-queues.ts +++ b/test/publisher/message-queues.ts @@ -341,10 +341,10 @@ describe('Message Queues', () => { spies = [sandbox.spy(), sandbox.spy()] as p.PublishCallback[]; }); - it('should begin another publish(drain) if there are pending batches', () => { + it('should begin another publish(drain) if there are pending batches', done => { const stub = sandbox.stub(queue, '_publish'); let once = false; - stub.callsFake((m, c, done) => { + stub.callsFake(async () => { if (!once) { // Drop in a second batch before calling the callback. const secondBatch = new FakeMessageBatch(); @@ -353,22 +353,23 @@ describe('Message Queues', () => { queue.batch = secondBatch; } once = true; - - done!(null); }); queue.batch = new FakeMessageBatch(); queue.batch.messages = fakeMessages; queue.batch.callbacks = spies; - queue.publishDrain(); - - assert.strictEqual(stub.callCount, 2); + queue.publishDrain().then(() => { + process.nextTick(() => { + assert.strictEqual(stub.callCount, 2); + done(); + }); + }); }); it('should not begin another publish(non-drain) if there are pending batches', () => { const stub = sandbox.stub(queue, '_publish'); let once = false; - stub.callsFake((m, c, done) => { + stub.callsFake(async () => { if (!once) { // Drop in a second batch before calling the callback. const secondBatch = new FakeMessageBatch(); @@ -377,28 +378,27 @@ describe('Message Queues', () => { queue.batch = secondBatch; } once = true; - - done!(null); }); queue.batch = new FakeMessageBatch(); queue.batch.messages = fakeMessages; queue.batch.callbacks = spies; - queue.publish(); - - assert.strictEqual(stub.callCount, 1); + queue.publish().then(() => { + assert.strictEqual(stub.callCount, 1); + }); }); - it('should emit "drain" if there is nothing left to publish', () => { + it('should emit "drain" if there is nothing left to publish', done => { const spy = sandbox.spy(); - sandbox - .stub(queue, '_publish') - .callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').callsFake(async () => {}); queue.on('drain', spy); - queue.publish(); - - assert.strictEqual(spy.callCount, 1); + queue.publish().then(() => { + process.nextTick(() => { + assert.strictEqual(spy.callCount, 1); + done(); + }); + }); }); }); }); From 8e18826f1c64672c02efe1ecdd229723638ce107 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 3 May 2023 16:08:30 -0400 Subject: [PATCH 30/52] feat: add modAck spans during leasing --- src/lease-manager.ts | 20 ++++++++++++++------ src/subscriber.ts | 14 ++++++++++++++ src/telemetry-tracing.ts | 4 ++++ test/lease-manager.ts | 2 ++ 4 files changed, 34 insertions(+), 6 deletions(-) diff --git a/src/lease-manager.ts b/src/lease-manager.ts index a3138aef2..a4f13d758 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -262,15 +262,23 @@ export class LeaseManager extends EventEmitter { const lifespan = (Date.now() - message.received) / (60 * 1000); if (lifespan < this._options.maxExtensionMinutes!) { + message.telemetrySub.modAckStart(); + if (this._subscriber.isExactlyOnceDelivery) { - message.modAckWithResponse(deadline).catch(e => { - // In the case of a permanent failure (temporary failures are retried), - // we need to stop trying to lease-manage the message. - message.ackFailed(e as AckError); - this.remove(message); - }); + message + .modAckWithResponse(deadline) + .catch(e => { + // In the case of a permanent failure (temporary failures are retried), + // we need to stop trying to lease-manage the message. + message.ackFailed(e as AckError); + this.remove(message); + }) + .finally(() => { + message.telemetrySub.modAckStop(); + }); } else { message.modAck(deadline); + message.telemetrySub.modAckStop(); } } else { this.remove(message); diff --git a/src/subscriber.ts b/src/subscriber.ts index 8c87d8b54..3b88c9755 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -89,6 +89,19 @@ export class SubscriberTelemetry { } } + modAckStart() { + if (!this.modAck) { + this.modAck = tracing.SpanMaker.createModAckSpan(this.parent); + } + } + + modAckStop() { + if (this.modAck) { + this.modAck.end(); + this.modAck = undefined; + } + } + schedulerStart() { if (!this.scheduler) { this.scheduler = tracing.SpanMaker.createReceiveSchedulerSpan( @@ -120,6 +133,7 @@ export class SubscriberTelemetry { } } + private modAck?: tracing.Span; private flow?: tracing.Span; private scheduler?: tracing.Span; private processing?: tracing.Span; diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 1d2e54f91..77e91f7fe 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -290,6 +290,10 @@ export class SpanMaker { return SpanMaker.createChildSpan(message, 'publish'); } + static createModAckSpan(message: MessageWithAttributes) { + return SpanMaker.createChildSpan(message, 'modify ack deadline'); + } + static createReceiveFlowSpan( message: MessageWithAttributes ): Span | undefined { diff --git a/test/lease-manager.ts b/test/lease-manager.ts index de5d633cf..8e9db34c7 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -52,6 +52,8 @@ class FakeSubscriberTelemetry { flowEnd() {} schedulerStart() {} schedulerEnd() {} + modAckStart() {} + modAckStop() {} processingStart() {} processingEnd() {} } From 569d7446a53fc2bda0d6265a255127efeb50a36b Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 3 May 2023 16:37:50 -0400 Subject: [PATCH 31/52] docs: add more code comment docs --- samples/typescript/openTelemetryTracing.ts | 6 +++++- src/publisher/index.ts | 3 ++- src/publisher/message-queues.ts | 2 ++ src/subscriber.ts | 15 ++++++++++++++- src/subscription.ts | 4 +++- 5 files changed, 26 insertions(+), 4 deletions(-) diff --git a/samples/typescript/openTelemetryTracing.ts b/samples/typescript/openTelemetryTracing.ts index bb68af87a..f163da3d4 100644 --- a/samples/typescript/openTelemetryTracing.ts +++ b/samples/typescript/openTelemetryTracing.ts @@ -54,8 +54,11 @@ import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; // Enable the diagnostic logger for OpenTelemetry diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); +// Log spans out to the console for this test const exporter = new ConsoleSpanExporter(); +// Build a tracer provider and a span processor to do +// something with the spans we're generating. const provider = new NodeTracerProvider({ resource: new Resource({ [SemanticResourceAttributes.SERVICE_NAME]: 'otel example', @@ -105,7 +108,8 @@ async function subscriptionListen(subscriptionNameOrId: string) { pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); - // Wait a bit for the subscription. For the sample only. + // Wait a bit for the subscription to receive messages. + // For the sample only. setTimeout(() => { pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); }, SUBSCRIBER_TIMEOUT * 1000); diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 47429255d..8a368f506 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -14,7 +14,6 @@ * limitations under the License. */ -import {promisify} from '@google-cloud/promisify'; import * as extend from 'extend'; import {CallOptions} from 'google-gax'; import {isSpanContextValid, Span} from '@opentelemetry/api'; @@ -210,6 +209,8 @@ export class Publisher { } } + // Ensure that there's a parent span for subsequent publishes + // to hang off of. this.getParentSpan(message); if (!message.orderingKey) { diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 7e71c5544..6388ecbe6 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -145,6 +145,8 @@ export abstract class MessageQueue extends EventEmitter { throw e; } finally { messages.forEach(m => { + // We're finished with both the RPC and the whole publish operation, + // so close out all of the related spans. m.telemetryRpc?.end(); m.telemetrySpan?.end(); }); diff --git a/src/subscriber.ts b/src/subscriber.ts index 3b88c9755..f8137e13a 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -63,7 +63,8 @@ export class AckError extends Error { } /** - * Tracks the various spans in receive telemetry. + * Tracks the various spans in receive telemetry. This is a little + * extra abstraction in case we want to allow more providers later. * * @private */ @@ -71,17 +72,20 @@ export class SubscriberTelemetry { parent: Message; sub: Subscriber; + // These are always attached to a message (and its subscriber). constructor(parent: Message, sub: Subscriber) { this.parent = parent; this.sub = sub; } + // Start a flow control span if needed. flowStart() { if (!this.flow) { this.flow = tracing.SpanMaker.createReceiveFlowSpan(this.parent); } } + // End any flow control span. flowEnd() { if (this.flow) { this.flow.end(); @@ -89,12 +93,14 @@ export class SubscriberTelemetry { } } + // Start a leasing modAck span if needed. modAckStart() { if (!this.modAck) { this.modAck = tracing.SpanMaker.createModAckSpan(this.parent); } } + // End any leasing modAck span. modAckStop() { if (this.modAck) { this.modAck.end(); @@ -102,6 +108,9 @@ export class SubscriberTelemetry { } } + // Start a scheduler span if needed. + // Note: This is not currently used in Node, because there is no + // scheduler process, due to the way messages are delivered one at a time. schedulerStart() { if (!this.scheduler) { this.scheduler = tracing.SpanMaker.createReceiveSchedulerSpan( @@ -110,6 +119,7 @@ export class SubscriberTelemetry { } } + // End any schedular span. schedulerEnd() { if (this.scheduler) { this.scheduler.end(); @@ -117,6 +127,8 @@ export class SubscriberTelemetry { } } + // Start a processing span if needed. + // This is for user processing, during on('message') delivery. processingStart(subName: string) { if (!this.processing) { this.processing = tracing.SpanMaker.createReceiveProcessSpan( @@ -126,6 +138,7 @@ export class SubscriberTelemetry { } } + // End any processing span. processingEnd() { if (this.processing) { this.processing.end(); diff --git a/src/subscription.ts b/src/subscription.ts index e44216010..1ac321d32 100644 --- a/src/subscription.ts +++ b/src/subscription.ts @@ -269,6 +269,8 @@ export type DetachSubscriptionResponse = EmptyResponse; * ``` */ export class Subscription extends WrappingEmitter { + // Note: WrappingEmitter is used here to wrap user processing callbacks. + // We do this to be able to build telemetry spans around them. pubsub: PubSub; iam: IAM; topic?: Topic | string; @@ -368,7 +370,7 @@ export class Subscription extends WrappingEmitter { // In that case, we need to tag on to their Promise to end the span. // Otherwise, the listener chain is sync, and we can close out sync. const result = listener(...args) as unknown as Promise; - if (!!result && typeof result.then === 'function') { + if (result && typeof result.then === 'function') { result.then(() => span?.end()); } else { span?.end(); From 886c0eec4da25619eefba309ed07aa110f18a87a Mon Sep 17 00:00:00 2001 From: Owl Bot Date: Wed, 3 May 2023 20:51:33 +0000 Subject: [PATCH 32/52] =?UTF-8?q?=F0=9F=A6=89=20Updates=20from=20OwlBot=20?= =?UTF-8?q?post-processor?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md --- samples/openTelemetryTracing.js | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index bed6c9bd2..311ddc2d8 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -60,8 +60,11 @@ const { // Enable the diagnostic logger for OpenTelemetry diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); +// Log spans out to the console for this test const exporter = new ConsoleSpanExporter(); +// Build a tracer provider and a span processor to do +// something with the spans we're generating. const provider = new NodeTracerProvider({ resource: new Resource({ [SemanticResourceAttributes.SERVICE_NAME]: 'otel example', @@ -111,7 +114,8 @@ async function subscriptionListen(subscriptionNameOrId) { pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); - // Wait a bit for the subscription. For the sample only. + // Wait a bit for the subscription to receive messages. + // For the sample only. setTimeout(() => { pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); }, SUBSCRIBER_TIMEOUT * 1000); From 8b7e0a9c06be00a0ec8e409de7692236ebe7b962 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 3 May 2023 17:52:20 -0400 Subject: [PATCH 33/52] tests: unit test fixes, also include deadline and initial parameters for modAck --- src/lease-manager.ts | 6 +++++- src/subscriber.ts | 20 ++++++++++++++++++-- src/telemetry-tracing.ts | 15 +++++++++++++-- test/subscriber.ts | 14 ++++++++++---- 4 files changed, 46 insertions(+), 9 deletions(-) diff --git a/src/lease-manager.ts b/src/lease-manager.ts index a4f13d758..1c05ad96f 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -17,6 +17,7 @@ import {EventEmitter} from 'events'; import {AckError, Message, Subscriber} from './subscriber'; import {defaultOptions} from './default-options'; +import {Duration} from './temporal'; export interface FlowControlOptions { allowExcessMessages?: boolean; @@ -262,7 +263,10 @@ export class LeaseManager extends EventEmitter { const lifespan = (Date.now() - message.received) / (60 * 1000); if (lifespan < this._options.maxExtensionMinutes!) { - message.telemetrySub.modAckStart(); + message.telemetrySub.modAckStart( + Duration.from({seconds: deadline}), + false + ); if (this._subscriber.isExactlyOnceDelivery) { message diff --git a/src/subscriber.ts b/src/subscriber.ts index f8137e13a..c051867fa 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -94,9 +94,13 @@ export class SubscriberTelemetry { } // Start a leasing modAck span if needed. - modAckStart() { + modAckStart(deadline: Duration, isInitial: boolean) { if (!this.modAck) { - this.modAck = tracing.SpanMaker.createModAckSpan(this.parent); + this.modAck = tracing.SpanMaker.createModAckSpan( + this.parent, + deadline, + isInitial + ); } } @@ -961,6 +965,10 @@ export class Subscriber extends EventEmitter { if (this.isExactlyOnceDelivery) { // For exactly-once delivery, we must validate that we got a valid // lease on the message before actually leasing it. + message.telemetrySub.modAckStart( + Duration.from({seconds: this.ackDeadline}), + true + ); message .modAckWithResponse(this.ackDeadline) .then(() => { @@ -970,9 +978,17 @@ export class Subscriber extends EventEmitter { // Temporary failures will retry, so if an error reaches us // here, that means a permanent failure. Silently drop these. this._discardMessage(message); + }) + .finally(() => { + message.telemetrySub.modAckStop(); }); } else { + message.telemetrySub.modAckStart( + Duration.from({seconds: this.ackDeadline}), + true + ); message.modAck(this.ackDeadline); + message.telemetrySub.modAckStop(); this._inventory.add(message); } } else { diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 77e91f7fe..052a90580 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -290,8 +290,19 @@ export class SpanMaker { return SpanMaker.createChildSpan(message, 'publish'); } - static createModAckSpan(message: MessageWithAttributes) { - return SpanMaker.createChildSpan(message, 'modify ack deadline'); + static createModAckSpan( + message: MessageWithAttributes, + deadline: Duration, + initial: boolean + ) { + const span = SpanMaker.createChildSpan(message, 'modify ack deadline'); + if (span) { + span.setAttributes({ + 'messaging.pubsub.modack_deadline_seconds': deadline.totalOf('second'), + 'messaging.pubsub.is_receipt_modack': initial ? 'true' : 'false', + } as unknown as Attributes); + } + return span; } static createReceiveFlowSpan( diff --git a/test/subscriber.ts b/test/subscriber.ts index 39a99c356..0eb377dde 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -759,6 +759,8 @@ describe('Subscriber', () => { }); it('should add messages to the inventory', done => { + const message = new Message(subscriber, RECEIVED_MESSAGE); + subscriber.open(); const modAckStub = sandbox.stub(subscriber, 'modAck'); @@ -772,7 +774,11 @@ describe('Subscriber', () => { // OTel is enabled during tests, so we need to delete the baggage. // eslint-disable-next-line @typescript-eslint/no-explicit-any - delete (addMsg as any).telemetrySpan; + const [addMsgAny, msgAny] = [addMsg as any, message as any]; + delete addMsgAny.telemetrySpan; + delete addMsgAny.telemetrySub; + delete msgAny.telemetrySpan; + delete msgAny.telemetrySub; assert.deepStrictEqual(addMsg, message); @@ -920,8 +926,8 @@ describe('Subscriber', () => { message.telemetrySpan?.end(); const spans = exporter.getFinishedSpans(); - assert.strictEqual(spans.length, 1); - const firstSpan = spans.concat().shift(); + assert.strictEqual(spans.length, 2); + const firstSpan = spans.pop(); assert.ok(firstSpan); assert.strictEqual(firstSpan.parentSpanId, parentSpanContext.spanId); assert.strictEqual( @@ -954,7 +960,7 @@ describe('Subscriber', () => { stream.emit('data', pullResponse); message.telemetrySpan?.end(); - assert.strictEqual(exporter.getFinishedSpans().length, 1); + assert.strictEqual(exporter.getFinishedSpans().length, 2); }); }); From 83830d68833799e1c92c810717be5433f88e80f1 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Mon, 8 May 2023 18:14:59 -0400 Subject: [PATCH 34/52] fix: make sure the subscriber span ends, update sample --- samples/package.json | 3 +- samples/typescript/openTelemetryTracing.ts | 50 ++++++++++++---------- src/lease-manager.ts | 3 ++ src/subscriber.ts | 10 +++++ src/telemetry-tracing.ts | 4 ++ test/subscriber.ts | 4 +- 6 files changed, 48 insertions(+), 26 deletions(-) diff --git a/samples/package.json b/samples/package.json index f082df517..790d5fbc7 100644 --- a/samples/package.json +++ b/samples/package.json @@ -21,11 +21,12 @@ "precompile": "npm run clean" }, "dependencies": { + "@google-cloud/opentelemetry-cloud-trace-exporter": "^2.0.0", "@google-cloud/pubsub": "^3.5.2", "@opentelemetry/api": "~1.1.0", "@opentelemetry/resources": "~1.3.1", - "@opentelemetry/sdk-trace-node": "~1.3.1", "@opentelemetry/sdk-trace-base": "~1.3.1", + "@opentelemetry/sdk-trace-node": "~1.3.1", "@opentelemetry/semantic-conventions": "~1.3.1", "avro-js": "^1.10.1", "p-defer": "^3.0.0", diff --git a/samples/typescript/openTelemetryTracing.ts b/samples/typescript/openTelemetryTracing.ts index f163da3d4..3672be85d 100644 --- a/samples/typescript/openTelemetryTracing.ts +++ b/samples/typescript/openTelemetryTracing.ts @@ -43,10 +43,13 @@ import {Message, PubSub} from '@google-cloud/pubsub'; import * as otel from '@opentelemetry/sdk-trace-node'; import {diag, DiagConsoleLogger, DiagLogLevel} from '@opentelemetry/api'; const {NodeTracerProvider} = otel; -import { - SimpleSpanProcessor, - ConsoleSpanExporter, -} from '@opentelemetry/sdk-trace-base'; +import {SimpleSpanProcessor} from '@opentelemetry/sdk-trace-base'; + +// To output to the console for testing, use the ConsoleSpanExporter. +// import {ConsoleSpanExporter} from '@opentelemetry/sdk-trace-base'; + +// To output to Cloud Trace, import the OpenTelemetry bridge library. +import {TraceExporter} from '@google-cloud/opentelemetry-cloud-trace-exporter'; import {Resource} from '@opentelemetry/resources'; import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; @@ -54,8 +57,11 @@ import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; // Enable the diagnostic logger for OpenTelemetry diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); -// Log spans out to the console for this test -const exporter = new ConsoleSpanExporter(); +// Log spans out to the console, for testing. +// const exporter = new ConsoleSpanExporter(); + +// Log spans out to Cloud Trace, for production. +const exporter = new TraceExporter(); // Build a tracer provider and a span processor to do // something with the spans we're generating. @@ -72,46 +78,44 @@ provider.register(); const pubSubClient = new PubSub(); async function publishMessage(topicNameOrId: string, data: string) { - // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) + // Publishes the message as a string, e.g. "Hello, world!" + // or JSON.stringify(someObject) const dataBuffer = Buffer.from(data); - const messageId = await pubSubClient - .topic(topicNameOrId) - .publishMessage({data: dataBuffer}); + const publisher = pubSubClient.topic(topicNameOrId); + const messageId = await publisher.publishMessage({data: dataBuffer}); console.log(`Message ${messageId} published.`); } async function subscriptionListen(subscriptionNameOrId: string) { + const subscriber = pubSubClient.subscription(subscriptionNameOrId); + // Message handler for subscriber - const messageHandler = (message: Message) => { + const messageHandler = async (message: Message) => { console.log(`Message ${message.id} received.`); message.ack(); // Ensure that all spans got flushed by the exporter console.log('Cleaning up OpenTelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); + await processor.forceFlush(); + await subscriber.close(); }; - const errorHandler = (error: Error) => { + const errorHandler = async (error: Error) => { console.log('Received error:', error); console.log('Cleaning up OpenTelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); + await processor.forceFlush(); + await subscriber.close(); }; // Listens for new messages from the topic - pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); - pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); + subscriber.on('message', messageHandler); + subscriber.on('error', errorHandler); // Wait a bit for the subscription to receive messages. // For the sample only. setTimeout(() => { - pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); + subscriber.removeAllListeners(); }, SUBSCRIBER_TIMEOUT * 1000); } // [END opentelemetry_tracing] diff --git a/src/lease-manager.ts b/src/lease-manager.ts index 1c05ad96f..d687009d5 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -159,6 +159,9 @@ export class LeaseManager extends EventEmitter { * @private */ remove(message: Message): void { + // The subscriber span ends when it leaves leasing. + message.endTelemetrySpan(); + if (!this._messages.has(message)) { return; } diff --git a/src/subscriber.ts b/src/subscriber.ts index c051867fa..abe5626f9 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -208,6 +208,16 @@ export class Message implements tracing.MessageWithAttributes { */ telemetrySpan?: tracing.Span; + /** + * @private + * + * Ends any open subscribe telemetry span. + */ + endTelemetrySpan() { + this.telemetrySpan?.end(); + delete this.telemetrySpan; + } + /** * @private * diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 052a90580..774cb52cc 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -355,6 +355,8 @@ export class SpanMaker { * Injects the trace context into a Pub/Sub message (or other object with * an 'attributes' object) for propagation. * + * This is for the publish side. + * * @private */ export function injectSpan( @@ -420,6 +422,8 @@ export function containsSpanContext(message: MessageWithAttributes): boolean { * an 'attributes' object) from a propagation, for receive processing. If no * context was present, create a new parent span. * + * This is for the receive side. + * * @private */ export function extractSpan( diff --git a/test/subscriber.ts b/test/subscriber.ts index 0eb377dde..9e3e87452 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -923,7 +923,7 @@ describe('Subscriber', () => { const msgStream = stubs.get('messageStream'); msgStream.emit('data', pullResponse); - message.telemetrySpan?.end(); + message.endTelemetrySpan(); const spans = exporter.getFinishedSpans(); assert.strictEqual(spans.length, 2); @@ -959,7 +959,7 @@ describe('Subscriber', () => { const stream: FakeMessageStream = stubs.get('messageStream'); stream.emit('data', pullResponse); - message.telemetrySpan?.end(); + message.endTelemetrySpan(); assert.strictEqual(exporter.getFinishedSpans().length, 2); }); }); From 094f08d40f9f485c0c39153578bfbe9b0b474f02 Mon Sep 17 00:00:00 2001 From: Owl Bot Date: Mon, 8 May 2023 22:17:10 +0000 Subject: [PATCH 35/52] =?UTF-8?q?=F0=9F=A6=89=20Updates=20from=20OwlBot=20?= =?UTF-8?q?post-processor?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md --- samples/openTelemetryTracing.js | 50 ++++++++++++++++++--------------- 1 file changed, 28 insertions(+), 22 deletions(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index 311ddc2d8..9317c067b 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -47,10 +47,15 @@ const {PubSub} = require('@google-cloud/pubsub'); const otel = require('@opentelemetry/sdk-trace-node'); const {diag, DiagConsoleLogger, DiagLogLevel} = require('@opentelemetry/api'); const {NodeTracerProvider} = otel; +const {SimpleSpanProcessor} = require('@opentelemetry/sdk-trace-base'); + +// To output to the console for testing, use the ConsoleSpanExporter. +// import {ConsoleSpanExporter} from '@opentelemetry/sdk-trace-base'; + +// To output to Cloud Trace, import the OpenTelemetry bridge library. const { - SimpleSpanProcessor, - ConsoleSpanExporter, -} = require('@opentelemetry/sdk-trace-base'); + TraceExporter, +} = require('@google-cloud/opentelemetry-cloud-trace-exporter'); const {Resource} = require('@opentelemetry/resources'); const { @@ -60,8 +65,11 @@ const { // Enable the diagnostic logger for OpenTelemetry diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); -// Log spans out to the console for this test -const exporter = new ConsoleSpanExporter(); +// Log spans out to the console, for testing. +// const exporter = new ConsoleSpanExporter(); + +// Log spans out to Cloud Trace, for production. +const exporter = new TraceExporter(); // Build a tracer provider and a span processor to do // something with the spans we're generating. @@ -78,46 +86,44 @@ provider.register(); const pubSubClient = new PubSub(); async function publishMessage(topicNameOrId, data) { - // Publishes the message as a string, e.g. "Hello, world!" or JSON.stringify(someObject) + // Publishes the message as a string, e.g. "Hello, world!" + // or JSON.stringify(someObject) const dataBuffer = Buffer.from(data); - const messageId = await pubSubClient - .topic(topicNameOrId) - .publishMessage({data: dataBuffer}); + const publisher = pubSubClient.topic(topicNameOrId); + const messageId = await publisher.publishMessage({data: dataBuffer}); console.log(`Message ${messageId} published.`); } async function subscriptionListen(subscriptionNameOrId) { + const subscriber = pubSubClient.subscription(subscriptionNameOrId); + // Message handler for subscriber - const messageHandler = message => { + const messageHandler = async message => { console.log(`Message ${message.id} received.`); message.ack(); // Ensure that all spans got flushed by the exporter console.log('Cleaning up OpenTelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); + await processor.forceFlush(); + await subscriber.close(); }; - const errorHandler = error => { + const errorHandler = async error => { console.log('Received error:', error); console.log('Cleaning up OpenTelemetry exporter...'); - exporter.shutdown().then(() => { - // Cleaned up exporter. - process.exit(0); - }); + await processor.forceFlush(); + await subscriber.close(); }; // Listens for new messages from the topic - pubSubClient.subscription(subscriptionNameOrId).on('message', messageHandler); - pubSubClient.subscription(subscriptionNameOrId).on('error', errorHandler); + subscriber.on('message', messageHandler); + subscriber.on('error', errorHandler); // Wait a bit for the subscription to receive messages. // For the sample only. setTimeout(() => { - pubSubClient.subscription(subscriptionNameOrId).removeAllListeners(); + subscriber.removeAllListeners(); }, SUBSCRIBER_TIMEOUT * 1000); } // [END opentelemetry_tracing] From a713a2e9226a9d1b1e8706288ba0e1430b1b6b39 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 9 May 2023 12:46:17 -0400 Subject: [PATCH 36/52] tests: update unit test for latest changes --- test/lease-manager.ts | 1 + 1 file changed, 1 insertion(+) diff --git a/test/lease-manager.ts b/test/lease-manager.ts index 8e9db34c7..359caa833 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -71,6 +71,7 @@ class FakeMessage { return AckResponses.Success; } ackFailed() {} + endTelemetrySpan() {} } interface LeaseManagerInternals { From 171c8f987e27d89f8d351f844ec71d171696e8f9 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 9 May 2023 17:13:16 -0400 Subject: [PATCH 37/52] chore: pull promise-based publish out of OTel change --- src/publisher/index.ts | 7 +- src/publisher/message-queues.ts | 134 +++++++++++++++--------------- test/publisher/index.ts | 62 ++++++-------- test/publisher/message-queues.ts | 137 +++++++++++++++---------------- 4 files changed, 163 insertions(+), 177 deletions(-) diff --git a/src/publisher/index.ts b/src/publisher/index.ts index dfe340da8..12a81f91d 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -123,14 +123,12 @@ export class Publisher { // event listeners after we've completed flush(). q.removeListener('drain', flushResolver); }; - return q.on('drain', flushResolver); + q.on('drain', flushResolver); }) ) ); - const allPublishes = Promise.all( - toDrain.map(q => promisify(q.publishDrain).bind(q)()) - ); + const allPublishes = Promise.all(toDrain.map(q => q.publishDrain())); allPublishes .then(() => allDrains) @@ -139,6 +137,7 @@ export class Publisher { }) .catch(definedCallback); } + /** * Publish the provided message. * diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index ff3052cef..5ec3ed961 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -22,9 +22,7 @@ import {PublishError} from './publish-error'; import {Publisher, PubsubMessage, PublishCallback} from './'; import {google} from '../../protos/protos'; -export interface PublishDone { - (err: ServiceError | null): void; -} +import {promisify} from 'util'; /** * Queues are used to manage publishing batches of messages. @@ -73,7 +71,7 @@ export abstract class MessageQueue extends EventEmitter { * * @abstract */ - abstract publish(): void; + abstract publish(): Promise; /** * Method to finalize publishing. Does as many publishes as are needed @@ -81,49 +79,47 @@ export abstract class MessageQueue extends EventEmitter { * * @abstract */ - abstract publishDrain(): void; + abstract publishDrain(): Promise; /** * Accepts a batch of messages and publishes them to the API. * * @param {object[]} messages The messages to publish. * @param {PublishCallback[]} callbacks The corresponding callback functions. - * @param {function} [callback] Callback to be fired when publish is done. */ - _publish( + async _publish( messages: PubsubMessage[], - callbacks: PublishCallback[], - callback?: PublishDone - ): void { + callbacks: PublishCallback[] + ): Promise { const {topic, settings} = this.publisher; const reqOpts = { topic: topic.name, messages, }; if (messages.length === 0) { - if (typeof callback === 'function') { - // Do this on the next tick to avoid Zalgo with the publish request below. - process.nextTick(() => callback(null)); - } return; } - topic.request( - { + const requestCallback = topic.request; + const request = promisify(requestCallback.bind(topic)); + try { + const resp = await request({ client: 'PublisherClient', method: 'publish', reqOpts, gaxOpts: settings.gaxOpts!, - }, - (err, resp) => { - const messageIds = (resp && resp.messageIds) || []; - callbacks.forEach((callback, i) => callback(err, messageIds[i])); - - if (typeof callback === 'function') { - callback(err); - } + }); + + if (resp) { + const messageIds = resp.messageIds || []; + callbacks.forEach((callback, i) => callback(null, messageIds[i])); } - ); + } catch (e) { + const err = e as ServiceError; + callbacks.forEach(callback => callback(err)); + + throw e; + } } } @@ -156,16 +152,21 @@ export class Queue extends MessageQueue { */ add(message: PubsubMessage, callback: PublishCallback): void { if (!this.batch.canFit(message)) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } this.batch.add(message, callback); if (this.batch.isFull()) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } else if (!this.pending) { const {maxMilliseconds} = this.batchOptions; - this.pending = setTimeout(() => this.publish(), maxMilliseconds!); + this.pending = setTimeout(() => { + // Ignore errors. + this.publish().catch(() => {}); + }, maxMilliseconds!); } } @@ -176,8 +177,8 @@ export class Queue extends MessageQueue { * * @emits Queue#drain when all messages are sent. */ - publishDrain(callback?: PublishDone): void { - this._publishInternal(true, callback); + async publishDrain(): Promise { + await this._publishInternal(true); } /** @@ -185,8 +186,8 @@ export class Queue extends MessageQueue { * * Does _not_ attempt to further drain after one batch is sent. */ - publish(callback?: PublishDone): void { - this._publishInternal(false, callback); + async publish(): Promise { + await this._publishInternal(false); } /** @@ -194,8 +195,7 @@ export class Queue extends MessageQueue { * * @emits Queue#drain when all messages are sent. */ - _publishInternal(fullyDrain: boolean, callback?: PublishDone): void { - const definedCallback = callback || (() => {}); + async _publishInternal(fullyDrain: boolean): Promise { const {messages, callbacks} = this.batch; this.batch = new MessageBatch(this.batchOptions); @@ -205,21 +205,17 @@ export class Queue extends MessageQueue { delete this.pending; } - this._publish(messages, callbacks, (err: null | ServiceError) => { - if (err) { - definedCallback(err); - } else if (this.batch.messages.length) { - // We only do the indefinite go-arounds when we're trying to do a - // final drain for flush(). In all other cases, we want to leave - // subsequent batches alone so that they can time out as needed. - if (fullyDrain) { - this._publishInternal(true, callback); - } - } else { - this.emit('drain'); - definedCallback(null); + await this._publish(messages, callbacks); + if (this.batch.messages.length) { + // We only do the indefinite go-arounds when we're trying to do a + // final drain for flush(). In all other cases, we want to leave + // subsequent batches alone so that they can time out as needed. + if (fullyDrain) { + await this._publishInternal(true); } - }); + } else { + this.emit('drain'); + } } } @@ -286,7 +282,8 @@ export class OrderedQueue extends MessageQueue { } if (!this.currentBatch.canFit(message)) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } this.currentBatch.add(message, callback); @@ -295,7 +292,8 @@ export class OrderedQueue extends MessageQueue { // check again here if (!this.inFlight) { if (this.currentBatch.isFull()) { - this.publish(); + // Ignore errors. + this.publish().catch(() => {}); } else if (!this.pending) { this.beginNextPublish(); } @@ -309,7 +307,10 @@ export class OrderedQueue extends MessageQueue { const timeWaiting = Date.now() - this.currentBatch.created; const delay = Math.max(0, maxMilliseconds - timeWaiting); - this.pending = setTimeout(() => this.publish(), delay); + this.pending = setTimeout(() => { + // Ignore errors. + this.publish().catch(() => {}); + }, delay); } /** * Creates a new {@link MessageBatch} instance. @@ -344,8 +345,7 @@ export class OrderedQueue extends MessageQueue { * * @fires OrderedQueue#drain */ - publish(callback?: PublishDone): void { - const definedCallback = callback || (() => {}); + async publish(): Promise { this.inFlight = true; if (this.pending) { @@ -355,19 +355,21 @@ export class OrderedQueue extends MessageQueue { const {messages, callbacks} = this.batches.pop()!; - this._publish(messages, callbacks, (err: null | ServiceError) => { + try { + await this._publish(messages, callbacks); + } catch (e) { + const err = e as ServiceError; + this.inFlight = false; + this.handlePublishFailure(err); + } finally { this.inFlight = false; + } - if (err) { - this.handlePublishFailure(err); - definedCallback(err); - } else if (this.batches.length) { - this.beginNextPublish(); - } else { - this.emit('drain'); - definedCallback(null); - } - }); + if (this.batches.length) { + this.beginNextPublish(); + } else { + this.emit('drain'); + } } /** @@ -375,8 +377,8 @@ export class OrderedQueue extends MessageQueue { * * @fires OrderedQueue#drain */ - publishDrain(callback?: PublishDone): void { - this.publish(callback); + async publishDrain(): Promise { + await this.publish(); } /** diff --git a/test/publisher/index.ts b/test/publisher/index.ts index 82ba5d618..451bb82f2 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -60,19 +60,17 @@ class FakeQueue extends EventEmitter { updateOptions() {} // eslint-disable-next-line @typescript-eslint/no-unused-vars add(message: p.PubsubMessage, callback: p.PublishCallback): void {} - publish(callback: (err: Error | null) => void) { - this._publish([], [], callback); + async publish() { + await this._publish([], []); } - publishDrain(callback: (err: Error | null) => void) { - this.publish(callback); + async publishDrain() { + await this.publish(); } - _publish( + async _publish( // eslint-disable-next-line @typescript-eslint/no-unused-vars messages: p.PubsubMessage[], // eslint-disable-next-line @typescript-eslint/no-unused-vars - callbacks: p.PublishCallback[], - // eslint-disable-next-line @typescript-eslint/no-unused-vars - callback?: q.PublishDone + callbacks: p.PublishCallback[] ) {} } @@ -84,20 +82,17 @@ class FakeOrderedQueue extends FakeQueue { this.orderingKey = key; } resumePublishing(): void {} - // eslint-disable-next-line @typescript-eslint/no-unused-vars - publish(callback: (err: Error | null) => void) { - this._publish([], [], callback); + async publish() { + await this._publish([], []); } - publishDrain(callback: (err: Error | null) => void) { - this.publish(callback); + async publishDrain() { + await this.publish(); } - _publish( + async _publish( // eslint-disable-next-line @typescript-eslint/no-unused-vars messages: p.PubsubMessage[], // eslint-disable-next-line @typescript-eslint/no-unused-vars - callbacks: p.PublishCallback[], - // eslint-disable-next-line @typescript-eslint/no-unused-vars - callback?: q.PublishDone + callbacks: p.PublishCallback[] ) {} } @@ -350,20 +345,17 @@ describe('Publisher', () => { it('should drain any ordered queues on flush', done => { // We have to stub out the regular queue as well, so that the flush() operation finishes. - sandbox - .stub(FakeQueue.prototype, '_publish') - .callsFake((messages, callbacks, callback) => { - // Simulate the drain taking longer than the publishes. This can - // happen if more messages are queued during the publish(). - process.nextTick(() => { - publisher.queue.emit('drain'); - }); - if (typeof callback === 'function') callback(null); + sandbox.stub(FakeQueue.prototype, '_publish').callsFake(async () => { + // Simulate the drain taking longer than the publishes. This can + // happen if more messages are queued during the publish(). + process.nextTick(() => { + publisher.queue.emit('drain'); }); + }); sandbox .stub(FakeOrderedQueue.prototype, '_publish') - .callsFake((messages, callbacks, callback) => { + .callsFake(async () => { const queue = publisher.orderedQueues.get( orderingKey ) as unknown as FakeOrderedQueue; @@ -373,7 +365,6 @@ describe('Publisher', () => { process.nextTick(() => { queue.emit('drain'); }); - if (typeof callback === 'function') callback(null); }); publisher.orderedQueues.clear(); @@ -508,16 +499,13 @@ describe('Publisher', () => { describe('flush', () => { // The ordered queue drain test is above with the ordered queue tests. it('should drain the main publish queue', done => { - sandbox - .stub(publisher.queue, '_publish') - .callsFake((messages, callbacks, callback) => { - // Simulate the drain taking longer than the publishes. This can - // happen if more messages are queued during the publish(). - process.nextTick(() => { - publisher.queue.emit('drain'); - }); - if (typeof callback === 'function') callback(null); + sandbox.stub(publisher.queue, '_publish').callsFake(async () => { + // Simulate the drain taking longer than the publishes. This can + // happen if more messages are queued during the publish(). + process.nextTick(() => { + publisher.queue.emit('drain'); }); + }); publisher.flush(err => { assert.strictEqual(err, null); diff --git a/test/publisher/message-queues.ts b/test/publisher/message-queues.ts index eda38f1fa..f28b4e33b 100644 --- a/test/publisher/message-queues.ts +++ b/test/publisher/message-queues.ts @@ -169,42 +169,41 @@ describe('Message Queues', () => { assert.strictEqual(gaxOpts, callOptions); }); - it('should pass back any request errors', done => { + it('should pass back any request errors', async () => { const error = new Error('err') as ServiceError; sandbox.stub(topic, 'request').callsFake((config, callback) => { callback(error); }); - queue._publish(messages, callbacks, err => { + try { + await queue._publish(messages, callbacks); + assert.strictEqual(null, error, '_publish did not throw'); + } catch (e) { + const err = e as ServiceError; + assert.strictEqual(err, error); callbacks.forEach(callback => { const [err] = callback.lastCall.args; assert.strictEqual(err, error); }); - - done(); - }); + } }); - it('should pass back message ids', done => { + it('should pass back message ids', async () => { const messageIds = messages.map((_, i) => `message${i}`); sandbox.stub(topic, 'request').callsFake((config, callback) => { callback(null, {messageIds}); }); - queue._publish(messages, callbacks, err => { - assert.ifError(err); + await queue._publish(messages, callbacks); - callbacks.forEach((callback, i) => { - const [, messageId] = callback.lastCall.args; - const expectedId = `message${i}`; - assert.strictEqual(messageId, expectedId); - }); - - done(); + callbacks.forEach((callback, i) => { + const [, messageId] = callback.lastCall.args; + const expectedId = `message${i}`; + assert.strictEqual(messageId, expectedId); }); }); }); @@ -241,20 +240,19 @@ describe('Message Queues', () => { const addStub = sandbox.stub(queue.batch, 'add'); sandbox.stub(queue.batch, 'canFit').returns(false); - sandbox - .stub(queue, 'publish') - .onCall(0) - .callsFake(() => { - assert.strictEqual(addStub.callCount, 0); - done(); - }); + const publishStub = sandbox.stub(queue, 'publish'); + publishStub.onCall(0).callsFake(async () => { + assert.strictEqual(addStub.callCount, 0); + done(); + }); + publishStub.resolves(); queue.add(fakeMessage, spy); }); it('should add the message to the batch', () => { const stub = sandbox.stub(queue.batch, 'add'); - sandbox.stub(queue, 'publish'); + sandbox.stub(queue, 'publish').resolves(); queue.add(fakeMessage, spy); @@ -264,7 +262,7 @@ describe('Message Queues', () => { }); it('should publish immediately if the batch became full', () => { - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); sandbox.stub(queue.batch, 'isFull').returns(true); queue.add(fakeMessage, spy); @@ -274,7 +272,7 @@ describe('Message Queues', () => { it('should set a timeout to publish if need be', () => { const clock = sandbox.useFakeTimers(); - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); const maxMilliseconds = 1234; queue.batchOptions = {maxMilliseconds}; @@ -288,7 +286,7 @@ describe('Message Queues', () => { it('should noop if a timeout is already set', () => { const clock = sandbox.useFakeTimers(); - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); const maxMilliseconds = 1234; queue.batchOptions = {maxMilliseconds}; @@ -343,10 +341,10 @@ describe('Message Queues', () => { spies = [sandbox.spy(), sandbox.spy()] as p.PublishCallback[]; }); - it('should begin another publish(drain) if there are pending batches', () => { + it('should begin another publish(drain) if there are pending batches', done => { const stub = sandbox.stub(queue, '_publish'); let once = false; - stub.callsFake((m, c, done) => { + stub.callsFake(async () => { if (!once) { // Drop in a second batch before calling the callback. const secondBatch = new FakeMessageBatch(); @@ -355,22 +353,23 @@ describe('Message Queues', () => { queue.batch = secondBatch; } once = true; - - done!(null); }); queue.batch = new FakeMessageBatch(); queue.batch.messages = fakeMessages; queue.batch.callbacks = spies; - queue.publishDrain(); - - assert.strictEqual(stub.callCount, 2); + queue.publishDrain().then(() => { + process.nextTick(() => { + assert.strictEqual(stub.callCount, 2); + done(); + }); + }); }); it('should not begin another publish(non-drain) if there are pending batches', () => { const stub = sandbox.stub(queue, '_publish'); let once = false; - stub.callsFake((m, c, done) => { + stub.callsFake(async () => { if (!once) { // Drop in a second batch before calling the callback. const secondBatch = new FakeMessageBatch(); @@ -379,28 +378,27 @@ describe('Message Queues', () => { queue.batch = secondBatch; } once = true; - - done!(null); }); queue.batch = new FakeMessageBatch(); queue.batch.messages = fakeMessages; queue.batch.callbacks = spies; - queue.publish(); - - assert.strictEqual(stub.callCount, 1); + queue.publish().then(() => { + assert.strictEqual(stub.callCount, 1); + }); }); - it('should emit "drain" if there is nothing left to publish', () => { + it('should emit "drain" if there is nothing left to publish', done => { const spy = sandbox.spy(); - sandbox - .stub(queue, '_publish') - .callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').callsFake(async () => {}); queue.on('drain', spy); - queue.publish(); - - assert.strictEqual(spy.callCount, 1); + queue.publish().then(() => { + process.nextTick(() => { + assert.strictEqual(spy.callCount, 1); + done(); + }); + }); }); }); }); @@ -506,13 +504,12 @@ describe('Message Queues', () => { const addStub = sandbox.stub(batch, 'add'); sandbox.stub(batch, 'canFit').withArgs(fakeMessage).returns(false); - sandbox - .stub(queue, 'publish') - .onCall(0) - .callsFake(() => { - assert.strictEqual(addStub.callCount, 0); - done(); - }); + const publishStub = sandbox.stub(queue, 'publish'); + publishStub.onCall(0).callsFake(async () => { + assert.strictEqual(addStub.callCount, 0); + done(); + }); + publishStub.resolves(); queue.add(fakeMessage, spy); }); @@ -528,12 +525,12 @@ describe('Message Queues', () => { }); it('should noop after adding if a publish was triggered', () => { - const publishStub = sandbox.stub(queue, 'publish'); + const publishStub = sandbox.stub(queue, 'publish').resolves(); const beginPublishStub = sandbox.stub(queue, 'beginNextPublish'); sandbox.stub(batch, 'canFit').returns(false); - publishStub.onCall(0).callsFake(() => { + publishStub.onCall(0).callsFake(async () => { queue.inFlight = true; }); @@ -544,7 +541,7 @@ describe('Message Queues', () => { }); it('should publish immediately if the batch is full', () => { - const stub = sandbox.stub(queue, 'publish'); + const stub = sandbox.stub(queue, 'publish').resolves(); sandbox.stub(batch, 'isFull').returns(true); queue.add(fakeMessage, spy); @@ -585,14 +582,14 @@ describe('Message Queues', () => { }); it('should set a timeout that will call publish', done => { - sandbox.stub(queue, 'publish').callsFake(done); + sandbox.stub(queue, 'publish').callsFake(async () => done()); queue.beginNextPublish(); clock.tick(maxMilliseconds); }); it('should factor in the time the batch has been sitting', done => { const halfway = maxMilliseconds / 2; - sandbox.stub(queue, 'publish').callsFake(done); + sandbox.stub(queue, 'publish').callsFake(async () => done()); queue.currentBatch.created = Date.now() - halfway; queue.beginNextPublish(); clock.tick(halfway); @@ -689,46 +686,46 @@ describe('Message Queues', () => { assert.strictEqual(callbacks, spies); }); - it('should set inFlight to false after publishing', () => { - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + it('should set inFlight to false after publishing', async () => { + sandbox.stub(queue, '_publish').resolves(); - queue.publish(); + await queue.publish(); assert.strictEqual(queue.inFlight, false); }); - it('should handle any publish failures', () => { + it('should handle any publish failures', async () => { const error = new Error('err') as ServiceError; const stub = sandbox.stub(queue, 'handlePublishFailure'); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(error)); + sandbox.stub(queue, '_publish').rejects(error); - queue.publish(); + await queue.publish(); const [err] = stub.lastCall.args; assert.strictEqual(err, error); }); - it('should begin another publish if there are pending batches', () => { + it('should begin another publish if there are pending batches', async () => { const stub = sandbox.stub(queue, 'beginNextPublish'); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').resolves(); const secondBatch = new FakeMessageBatch(); secondBatch.messages = fakeMessages; secondBatch.callbacks = spies; queue.batches.push(secondBatch as b.MessageBatch); - queue.publish(); + await queue.publish(); assert.strictEqual(stub.callCount, 1); }); - it('should emit "drain" if there is nothing left to publish', () => { + it('should emit "drain" if there is nothing left to publish', async () => { const spy = sandbox.spy(); - sandbox.stub(queue, '_publish').callsFake((m, c, done) => done!(null)); + sandbox.stub(queue, '_publish').resolves(); queue.on('drain', spy); - queue.publish(); + await queue.publish(); assert.strictEqual(spy.callCount, 1); }); From 50bd1bfc0d9dc81537b332b8b8973f28be4db73c Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Mon, 12 Jun 2023 15:18:04 -0400 Subject: [PATCH 38/52] tests: update otel sample test --- samples/openTelemetryTracing.js | 10 ++++------ samples/system-test/openTelemetryTracing.test.ts | 2 ++ samples/typescript/openTelemetryTracing.ts | 8 ++++---- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/samples/openTelemetryTracing.js b/samples/openTelemetryTracing.js index 9317c067b..69107bd7f 100644 --- a/samples/openTelemetryTracing.js +++ b/samples/openTelemetryTracing.js @@ -50,12 +50,10 @@ const {NodeTracerProvider} = otel; const {SimpleSpanProcessor} = require('@opentelemetry/sdk-trace-base'); // To output to the console for testing, use the ConsoleSpanExporter. -// import {ConsoleSpanExporter} from '@opentelemetry/sdk-trace-base'; +const {ConsoleSpanExporter} = require('@opentelemetry/sdk-trace-base'); // To output to Cloud Trace, import the OpenTelemetry bridge library. -const { - TraceExporter, -} = require('@google-cloud/opentelemetry-cloud-trace-exporter'); +// import {TraceExporter} from '@google-cloud/opentelemetry-cloud-trace-exporter'; const {Resource} = require('@opentelemetry/resources'); const { @@ -66,10 +64,10 @@ const { diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); // Log spans out to the console, for testing. -// const exporter = new ConsoleSpanExporter(); +const exporter = new ConsoleSpanExporter(); // Log spans out to Cloud Trace, for production. -const exporter = new TraceExporter(); +// const exporter = new TraceExporter(); // Build a tracer provider and a span processor to do // something with the spans we're generating. diff --git a/samples/system-test/openTelemetryTracing.test.ts b/samples/system-test/openTelemetryTracing.test.ts index 90d101ab5..6713a5f07 100644 --- a/samples/system-test/openTelemetryTracing.test.ts +++ b/samples/system-test/openTelemetryTracing.test.ts @@ -49,6 +49,8 @@ describe('openTelemetry', () => { ); assert.match(stdout, /Message .* published./); assert.match(stdout, /Message .* received/); + assert.match(stdout, /send/); + assert.match(stdout, /receive/); assert.notMatch(stdout, /Received error/); }); }); diff --git a/samples/typescript/openTelemetryTracing.ts b/samples/typescript/openTelemetryTracing.ts index 3672be85d..c089f7492 100644 --- a/samples/typescript/openTelemetryTracing.ts +++ b/samples/typescript/openTelemetryTracing.ts @@ -46,10 +46,10 @@ const {NodeTracerProvider} = otel; import {SimpleSpanProcessor} from '@opentelemetry/sdk-trace-base'; // To output to the console for testing, use the ConsoleSpanExporter. -// import {ConsoleSpanExporter} from '@opentelemetry/sdk-trace-base'; +import {ConsoleSpanExporter} from '@opentelemetry/sdk-trace-base'; // To output to Cloud Trace, import the OpenTelemetry bridge library. -import {TraceExporter} from '@google-cloud/opentelemetry-cloud-trace-exporter'; +// import {TraceExporter} from '@google-cloud/opentelemetry-cloud-trace-exporter'; import {Resource} from '@opentelemetry/resources'; import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; @@ -58,10 +58,10 @@ import {SemanticResourceAttributes} from '@opentelemetry/semantic-conventions'; diag.setLogger(new DiagConsoleLogger(), DiagLogLevel.DEBUG); // Log spans out to the console, for testing. -// const exporter = new ConsoleSpanExporter(); +const exporter = new ConsoleSpanExporter(); // Log spans out to Cloud Trace, for production. -const exporter = new TraceExporter(); +// const exporter = new TraceExporter(); // Build a tracer provider and a span processor to do // something with the spans we're generating. From 0e7e5da9c079b3768aeb258c3f6c5b53172ccbc2 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 14 Jun 2023 15:00:48 -0400 Subject: [PATCH 39/52] wip: end all telemetry spans when clear() is called --- src/lease-manager.ts | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/lease-manager.ts b/src/lease-manager.ts index d687009d5..6d27a3978 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -130,6 +130,9 @@ export class LeaseManager extends EventEmitter { const wasFull = this.isFull(); this._pending = []; + this._messages.forEach(m => { + m.endTelemetrySpan(); + }); this._messages.clear(); this.bytes = 0; From a287386d4250c154d7c964de52a1ea68d4bbd625 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Wed, 14 Jun 2023 15:01:03 -0400 Subject: [PATCH 40/52] wip: add more unit tests --- test/iam.ts | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/test/iam.ts b/test/iam.ts index a2dab3ddc..d5f5763e2 100644 --- a/test/iam.ts +++ b/test/iam.ts @@ -81,10 +81,19 @@ describe('IAM', () => { assert.strictEqual(iam.request, fakeRequest); }); - it('should localize the ID', () => { + it('should localize the ID string', () => { assert.strictEqual(iam.id, ID); }); + it('should localize the ID getter', () => { + iam = new IAM(PUBSUB, { + get name() { + return 'test'; + }, + }); + assert.strictEqual(iam.id, 'test'); + }); + it('should promisify some of the things', () => { assert(promisified); }); From 0f2dc418372f2732d0ae29eeff29ac1827efeaa7 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:38:45 -0400 Subject: [PATCH 41/52] fix: include batch size with each published message span --- src/publisher/message-queues.ts | 2 +- src/telemetry-tracing.ts | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 27f089754..c9fb86f49 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -118,7 +118,7 @@ export abstract class MessageQueue extends EventEmitter { } messages.forEach(m => { - const span = tracing.SpanMaker.createPublishRpcSpan(m); + const span = tracing.SpanMaker.createPublishRpcSpan(m, messages.length); if (span) { m.telemetryRpc = span; } diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 774cb52cc..1bedd981f 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -286,8 +286,11 @@ export class SpanMaker { return SpanMaker.createChildSpan(message, 'publish scheduler'); } - static createPublishRpcSpan(message: PubsubMessage): Span | undefined { - return SpanMaker.createChildSpan(message, 'publish'); + static createPublishRpcSpan(message: PubsubMessage, messageCount: number): Span | undefined { + const span = SpanMaker.createChildSpan(message, 'publish'); + span?.setAttribute('messaging.pubsub.num_messages_in_batch', messageCount); + + return span; } static createModAckSpan( From 1ba495afba805d42b0db2d797959803aac838750 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:40:35 -0400 Subject: [PATCH 42/52] chore: fix lint --- src/telemetry-tracing.ts | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 1bedd981f..e43af9234 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -286,7 +286,10 @@ export class SpanMaker { return SpanMaker.createChildSpan(message, 'publish scheduler'); } - static createPublishRpcSpan(message: PubsubMessage, messageCount: number): Span | undefined { + static createPublishRpcSpan( + message: PubsubMessage, + messageCount: number + ): Span | undefined { const span = SpanMaker.createChildSpan(message, 'publish'); span?.setAttribute('messaging.pubsub.num_messages_in_batch', messageCount); From 49b0bc3fd98de45d7319921c45baa79c803bd24c Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:45:57 -0400 Subject: [PATCH 43/52] fix: separate out the getter and setter classes --- src/telemetry-tracing.ts | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index e43af9234..7d01b0eff 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -120,11 +120,7 @@ export interface MessageWithAttributes { * * @private */ -export class PubsubMessageGetSet - implements - TextMapGetter, - TextMapSetter -{ +export class PubsubMessageGetSet { static keyPrefix = 'googclient_'; keys(carrier: MessageWithAttributes): string[] { @@ -133,17 +129,27 @@ export class PubsubMessageGetSet .map(n => n.substring(PubsubMessageGetSet.keyPrefix.length)); } - private attributeName(key: string): string { + protected attributeName(key: string): string { return `${PubsubMessageGetSet.keyPrefix}${key}`; } +} +export class PubsubMessageGet + extends PubsubMessageGetSet + implements TextMapGetter +{ get( carrier: MessageWithAttributes, key: string ): string | string[] | undefined { return carrier?.attributes?.[this.attributeName(key)]; } +} +export class PubsubMessageSet + extends PubsubMessageGetSet + implements TextMapSetter +{ set(carrier: MessageWithAttributes, key: string, value: string): void { if (!carrier.attributes) { carrier.attributes = {}; @@ -157,14 +163,14 @@ export class PubsubMessageGetSet * * @private */ -export const pubsubGetter = new PubsubMessageGetSet(); +export const pubsubGetter = new PubsubMessageGet(); /** * The setter to use when calling inject() on a Pub/Sub message. * * @private */ -export const pubsubSetter = pubsubGetter; +export const pubsubSetter = new PubsubMessageSet(); /** * Description of the data structure passed for span attributes. From 12f400deebdc8c3b7c94c46325cac97ca36753eb Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 17:15:11 -0400 Subject: [PATCH 44/52] chore: rename span related objects in subscriber --- src/lease-manager.ts | 17 +++++++---------- src/subscriber.ts | 29 +++++++++++++++-------------- test/subscriber.ts | 4 ++-- 3 files changed, 24 insertions(+), 26 deletions(-) diff --git a/src/lease-manager.ts b/src/lease-manager.ts index 6d27a3978..73043b765 100644 --- a/src/lease-manager.ts +++ b/src/lease-manager.ts @@ -105,7 +105,7 @@ export class LeaseManager extends EventEmitter { this._messages.add(message); this.bytes += message.length; - message.telemetrySub.flowStart(); + message.subSpans.flowStart(); if (allowExcessMessages! || !wasFull) { this._dispense(message); @@ -131,7 +131,7 @@ export class LeaseManager extends EventEmitter { this._pending = []; this._messages.forEach(m => { - m.endTelemetrySpan(); + m.endParentSpan(); }); this._messages.clear(); this.bytes = 0; @@ -163,7 +163,7 @@ export class LeaseManager extends EventEmitter { */ remove(message: Message): void { // The subscriber span ends when it leaves leasing. - message.endTelemetrySpan(); + message.endParentSpan(); if (!this._messages.has(message)) { return; @@ -249,7 +249,7 @@ export class LeaseManager extends EventEmitter { */ private _dispense(message: Message): void { if (this._subscriber.isOpen) { - message.telemetrySub.flowEnd(); + message.subSpans.flowEnd(); process.nextTick(() => { this._subscriber.emit('message', message); }); @@ -269,10 +269,7 @@ export class LeaseManager extends EventEmitter { const lifespan = (Date.now() - message.received) / (60 * 1000); if (lifespan < this._options.maxExtensionMinutes!) { - message.telemetrySub.modAckStart( - Duration.from({seconds: deadline}), - false - ); + message.subSpans.modAckStart(Duration.from({seconds: deadline}), false); if (this._subscriber.isExactlyOnceDelivery) { message @@ -284,11 +281,11 @@ export class LeaseManager extends EventEmitter { this.remove(message); }) .finally(() => { - message.telemetrySub.modAckStop(); + message.subSpans.modAckStop(); }); } else { message.modAck(deadline); - message.telemetrySub.modAckStop(); + message.subSpans.modAckStop(); } } else { this.remove(message); diff --git a/src/subscriber.ts b/src/subscriber.ts index abe5626f9..4d968fe5d 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -199,31 +199,32 @@ export class Message implements tracing.MessageWithAttributes { /** * @private * - * Tracks any telemetry span through the library, on the receive side. This will - * be the original publisher-side span if we have one. + * Tracks a telemetry tracing parent span through the receive process. This will + * be the original publisher-side span if we have one; otherwise we'll create + * a "publisher" span to hang new subscriber spans onto. * * This needs to be declared explicitly here, because having a public class * implement a private interface seems to confuse TypeScript. (And it's needed * in unit tests.) */ - telemetrySpan?: tracing.Span; + parentSpan?: tracing.Span; /** * @private * - * Ends any open subscribe telemetry span. + * Ends any open subscribe telemetry tracing span. */ - endTelemetrySpan() { - this.telemetrySpan?.end(); - delete this.telemetrySpan; + endParentSpan() { + this.parentSpan?.end(); + delete this.parentSpan; } /** * @private * - * Tracks subscriber-specific telemetry spans through the library. + * Tracks subscriber-specific telemetry objects through the library. */ - telemetrySub: SubscriberTelemetry; + subSpans: SubscriberTelemetry; private _ackFailed?: AckError; @@ -309,7 +310,7 @@ export class Message implements tracing.MessageWithAttributes { * * @private */ - this.telemetrySub = new SubscriberTelemetry(this, sub); + this.subSpans = new SubscriberTelemetry(this, sub); this._handled = false; this._length = this.data.length; @@ -975,7 +976,7 @@ export class Subscriber extends EventEmitter { if (this.isExactlyOnceDelivery) { // For exactly-once delivery, we must validate that we got a valid // lease on the message before actually leasing it. - message.telemetrySub.modAckStart( + message.subSpans.modAckStart( Duration.from({seconds: this.ackDeadline}), true ); @@ -990,15 +991,15 @@ export class Subscriber extends EventEmitter { this._discardMessage(message); }) .finally(() => { - message.telemetrySub.modAckStop(); + message.subSpans.modAckStop(); }); } else { - message.telemetrySub.modAckStart( + message.subSpans.modAckStart( Duration.from({seconds: this.ackDeadline}), true ); message.modAck(this.ackDeadline); - message.telemetrySub.modAckStop(); + message.subSpans.modAckStop(); this._inventory.add(message); } } else { diff --git a/test/subscriber.ts b/test/subscriber.ts index 9e3e87452..5a2f87a03 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -923,7 +923,7 @@ describe('Subscriber', () => { const msgStream = stubs.get('messageStream'); msgStream.emit('data', pullResponse); - message.endTelemetrySpan(); + message.endParentSpan(); const spans = exporter.getFinishedSpans(); assert.strictEqual(spans.length, 2); @@ -959,7 +959,7 @@ describe('Subscriber', () => { const stream: FakeMessageStream = stubs.get('messageStream'); stream.emit('data', pullResponse); - message.endTelemetrySpan(); + message.endParentSpan(); assert.strictEqual(exporter.getFinishedSpans().length, 2); }); }); From febcd16b9bc28dbf9e4639cae932850a6dd91c1a Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 17:19:35 -0400 Subject: [PATCH 45/52] chore: rename SpanMaker --- src/publisher/flow-publisher.ts | 2 +- src/publisher/index.ts | 2 +- src/publisher/message-queues.ts | 6 +++--- src/subscriber.ts | 16 ++++++++-------- src/subscription.ts | 2 +- src/telemetry-tracing.ts | 22 +++++++++++----------- test/telemetry-tracing.ts | 8 ++++---- 7 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/publisher/flow-publisher.ts b/src/publisher/flow-publisher.ts index db18b0432..77c24c424 100644 --- a/src/publisher/flow-publisher.ts +++ b/src/publisher/flow-publisher.ts @@ -78,7 +78,7 @@ export class FlowControlledPublisher { */ publish(message: PubsubMessage): Promise | null { const flowSpan = message.telemetrySpan - ? tracing.SpanMaker.createPublishFlowSpan(message) + ? tracing.PubsubSpans.createPublishFlowSpan(message) : undefined; const doPublish = () => { flowSpan?.end(); diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 228374843..6ffd8538e 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -343,7 +343,7 @@ export class Publisher { return message.telemetrySpan; } - const span = tracing.SpanMaker.createPublisherSpan( + const span = tracing.PubsubSpans.createPublisherSpan( message, this.topic.name ); diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index c9fb86f49..9b14e3938 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -110,7 +110,7 @@ export abstract class MessageQueue extends EventEmitter { await topic.pubsub.getClientConfig(); } spanMessages.forEach(m => { - tracing.SpanMaker.updatePublisherTopicName( + tracing.PubsubSpans.updatePublisherTopicName( m.telemetrySpan!, topic.name ); @@ -118,7 +118,7 @@ export abstract class MessageQueue extends EventEmitter { } messages.forEach(m => { - const span = tracing.SpanMaker.createPublishRpcSpan(m, messages.length); + const span = tracing.PubsubSpans.createPublishRpcSpan(m, messages.length); if (span) { m.telemetryRpc = span; } @@ -190,7 +190,7 @@ export class Queue extends MessageQueue { } message.telemetryBatching = - tracing.SpanMaker.createPublishBatchSpan(message); + tracing.PubsubSpans.createPublishBatchSpan(message); this.batch.add(message, callback); diff --git a/src/subscriber.ts b/src/subscriber.ts index 4d968fe5d..20c015c97 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -81,7 +81,7 @@ export class SubscriberTelemetry { // Start a flow control span if needed. flowStart() { if (!this.flow) { - this.flow = tracing.SpanMaker.createReceiveFlowSpan(this.parent); + this.flow = tracing.PubsubSpans.createReceiveFlowSpan(this.parent); } } @@ -96,7 +96,7 @@ export class SubscriberTelemetry { // Start a leasing modAck span if needed. modAckStart(deadline: Duration, isInitial: boolean) { if (!this.modAck) { - this.modAck = tracing.SpanMaker.createModAckSpan( + this.modAck = tracing.PubsubSpans.createModAckSpan( this.parent, deadline, isInitial @@ -117,7 +117,7 @@ export class SubscriberTelemetry { // scheduler process, due to the way messages are delivered one at a time. schedulerStart() { if (!this.scheduler) { - this.scheduler = tracing.SpanMaker.createReceiveSchedulerSpan( + this.scheduler = tracing.PubsubSpans.createReceiveSchedulerSpan( this.parent ); } @@ -135,7 +135,7 @@ export class SubscriberTelemetry { // This is for user processing, during on('message') delivery. processingStart(subName: string) { if (!this.processing) { - this.processing = tracing.SpanMaker.createReceiveProcessSpan( + this.processing = tracing.PubsubSpans.createReceiveProcessSpan( this.parent, subName ); @@ -697,7 +697,7 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, true); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, true); // Ignore this in this version of the method (but hook catch // to avoid unhandled exceptions). @@ -723,7 +723,7 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, true); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, true); await this._acks.add(message); @@ -827,7 +827,7 @@ export class Subscriber extends EventEmitter { * @private */ async nack(message: Message): Promise { - const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, false); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, false); await this.modAck(message, 0); @@ -846,7 +846,7 @@ export class Subscriber extends EventEmitter { * @private */ async nackWithResponse(message: Message): Promise { - const ackSpan = tracing.SpanMaker.createReceiveResponseSpan(message, false); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, false); const response = await this.modAckWithResponse(message, 0); ackSpan?.end(); return response; diff --git a/src/subscription.ts b/src/subscription.ts index 1ac321d32..c12fd4981 100644 --- a/src/subscription.ts +++ b/src/subscription.ts @@ -361,7 +361,7 @@ export class Subscription extends WrappingEmitter { if (eventName !== 'message') { return listener(...args); } else { - const span = tracing.SpanMaker.createReceiveProcessSpan( + const span = tracing.PubsubSpans.createReceiveProcessSpan( args[0] as Message, this.name ); diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 7d01b0eff..9c809477a 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -211,7 +211,7 @@ export const modernAttributeName = 'googclient_traceparent'; */ export const legacyAttributeName = 'googclient_OpenTelemetrySpanContext'; -export class SpanMaker { +export class PubsubSpans { static createPublisherSpan(message: PubsubMessage, topicName: string): Span { const spanAttributes = { // Add Opentelemetry semantic convention attributes to the span, based on: @@ -285,18 +285,18 @@ export class SpanMaker { } static createPublishFlowSpan(message: PubsubMessage): Span | undefined { - return SpanMaker.createChildSpan(message, 'publisher flow control'); + return PubsubSpans.createChildSpan(message, 'publisher flow control'); } static createPublishBatchSpan(message: PubsubMessage): Span | undefined { - return SpanMaker.createChildSpan(message, 'publish scheduler'); + return PubsubSpans.createChildSpan(message, 'publish scheduler'); } static createPublishRpcSpan( message: PubsubMessage, messageCount: number ): Span | undefined { - const span = SpanMaker.createChildSpan(message, 'publish'); + const span = PubsubSpans.createChildSpan(message, 'publish'); span?.setAttribute('messaging.pubsub.num_messages_in_batch', messageCount); return span; @@ -307,7 +307,7 @@ export class SpanMaker { deadline: Duration, initial: boolean ) { - const span = SpanMaker.createChildSpan(message, 'modify ack deadline'); + const span = PubsubSpans.createChildSpan(message, 'modify ack deadline'); if (span) { span.setAttributes({ 'messaging.pubsub.modack_deadline_seconds': deadline.totalOf('second'), @@ -320,20 +320,20 @@ export class SpanMaker { static createReceiveFlowSpan( message: MessageWithAttributes ): Span | undefined { - return SpanMaker.createChildSpan(message, 'subscriber flow control'); + return PubsubSpans.createChildSpan(message, 'subscriber flow control'); } static createReceiveSchedulerSpan( message: MessageWithAttributes ): Span | undefined { - return SpanMaker.createChildSpan(message, 'subscribe scheduler'); + return PubsubSpans.createChildSpan(message, 'subscribe scheduler'); } static createReceiveProcessSpan( message: MessageWithAttributes, subName: string ): Span | undefined { - return SpanMaker.createChildSpan(message, `${subName} process`); + return PubsubSpans.createChildSpan(message, `${subName} process`); } static setReceiveProcessResult(span: Span, isAck: boolean) { @@ -345,7 +345,7 @@ export class SpanMaker { deadline: Duration, isInitial: boolean ): Span | undefined { - const span = SpanMaker.createChildSpan(message, 'modify ack deadline'); + const span = PubsubSpans.createChildSpan(message, 'modify ack deadline'); span?.setAttribute( 'messaging.pubsub.modack_deadline_seconds', deadline.totalOf('second') @@ -359,7 +359,7 @@ export class SpanMaker { isAck: boolean ): Span | undefined { const name = isAck ? 'ack' : 'nack'; - return SpanMaker.createChildSpan(message, name); + return PubsubSpans.createChildSpan(message, name); } } @@ -473,7 +473,7 @@ export function extractSpan( } } - const span = SpanMaker.createReceiveSpan(message, subName, context); + const span = PubsubSpans.createReceiveSpan(message, subName, context); message.telemetrySpan = span; return span; } diff --git a/test/telemetry-tracing.ts b/test/telemetry-tracing.ts index de450d580..69153957e 100644 --- a/test/telemetry-tracing.ts +++ b/test/telemetry-tracing.ts @@ -31,7 +31,7 @@ describe('OpenTelemetryTracer', () => { it('creates a span', () => { const message: PubsubMessage = {}; - const span = otel.SpanMaker.createPublisherSpan( + const span = otel.PubsubSpans.createPublisherSpan( message, 'test topic' ) as trace.Span; @@ -49,7 +49,7 @@ describe('OpenTelemetryTracer', () => { const message: PubsubMessage = { attributes: {}, }; - const span = otel.SpanMaker.createPublisherSpan( + const span = otel.PubsubSpans.createPublisherSpan( message, 'test topic' ) as trace.Span; @@ -68,7 +68,7 @@ describe('OpenTelemetryTracer', () => { const message: PubsubMessage = { attributes: {}, }; - const span = otel.SpanMaker.createPublisherSpan(message, 'test topic'); + const span = otel.PubsubSpans.createPublisherSpan(message, 'test topic'); otel.injectSpan(span, message, otel.OpenTelemetryLevel.Legacy); @@ -93,7 +93,7 @@ describe('OpenTelemetryTracer', () => { [otel.modernAttributeName]: 'bazbar', }, }; - const span = otel.SpanMaker.createPublisherSpan(message, 'test topic'); + const span = otel.PubsubSpans.createPublisherSpan(message, 'test topic'); const warnSpy = sinon.spy(console, 'warn'); try { From a4718757a2dccfc96ef2c27963bfadce42fd1811 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 17:22:34 -0400 Subject: [PATCH 46/52] docs: improved comments and lint fixes --- src/subscriber.ts | 23 ++++++++++++++++++----- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/src/subscriber.ts b/src/subscriber.ts index 20c015c97..c84692f10 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -511,7 +511,8 @@ export interface SubscriberOptions { useLegacyFlowControl?: boolean; streamingOptions?: MessageStreamOptions; - /** @deprecated Unset and use context propagation. */ + /** @deprecated Unset this and instantiate a tracer; support will be + * enabled automatically. */ enableOpenTelemetryTracing?: boolean; } @@ -697,7 +698,10 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, true); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan( + message, + true + ); // Ignore this in this version of the method (but hook catch // to avoid unhandled exceptions). @@ -723,7 +727,10 @@ export class Subscriber extends EventEmitter { const ackTimeSeconds = (Date.now() - message.received) / 1000; this.updateAckDeadline(ackTimeSeconds); - const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, true); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan( + message, + true + ); await this._acks.add(message); @@ -827,7 +834,10 @@ export class Subscriber extends EventEmitter { * @private */ async nack(message: Message): Promise { - const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, false); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan( + message, + false + ); await this.modAck(message, 0); @@ -846,7 +856,10 @@ export class Subscriber extends EventEmitter { * @private */ async nackWithResponse(message: Message): Promise { - const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan(message, false); + const ackSpan = tracing.PubsubSpans.createReceiveResponseSpan( + message, + false + ); const response = await this.modAckWithResponse(message, 0); ackSpan?.end(); return response; From a37194ded8d45285b78c97e7eb81b648fbb97d07 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Tue, 15 Aug 2023 17:26:57 -0400 Subject: [PATCH 47/52] docs: more class/comment updating --- src/subscriber.ts | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/subscriber.ts b/src/subscriber.ts index c84692f10..30f5eeb68 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -63,12 +63,11 @@ export class AckError extends Error { } /** - * Tracks the various spans in receive telemetry. This is a little - * extra abstraction in case we want to allow more providers later. + * Tracks the various spans related to subscriber/receive tracing. * * @private */ -export class SubscriberTelemetry { +export class SubscriberSpans { parent: Message; sub: Subscriber; @@ -224,7 +223,7 @@ export class Message implements tracing.MessageWithAttributes { * * Tracks subscriber-specific telemetry objects through the library. */ - subSpans: SubscriberTelemetry; + subSpans: SubscriberSpans; private _ackFailed?: AckError; @@ -306,11 +305,11 @@ export class Message implements tracing.MessageWithAttributes { this.received = Date.now(); /** - * Telemetry tracking objects. + * Telemetry tracing objects. * * @private */ - this.subSpans = new SubscriberTelemetry(this, sub); + this.subSpans = new SubscriberSpans(this, sub); this._handled = false; this._length = this.data.length; From 9ec1fdfde082755c00040e3a99f2499839adf01f Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Thu, 17 Aug 2023 16:09:17 -0400 Subject: [PATCH 48/52] fix: missing spans caused by renames; also do more renames --- src/publisher/flow-publisher.ts | 2 +- src/publisher/index.ts | 4 ++-- src/publisher/message-queues.ts | 18 +++++++----------- src/publisher/pubsub-message.ts | 8 ++++---- src/telemetry-tracing.ts | 31 +++++++++++++++++++++---------- test/publisher/index.ts | 2 +- 6 files changed, 36 insertions(+), 29 deletions(-) diff --git a/src/publisher/flow-publisher.ts b/src/publisher/flow-publisher.ts index 77c24c424..74e3b3279 100644 --- a/src/publisher/flow-publisher.ts +++ b/src/publisher/flow-publisher.ts @@ -77,7 +77,7 @@ export class FlowControlledPublisher { * ``` */ publish(message: PubsubMessage): Promise | null { - const flowSpan = message.telemetrySpan + const flowSpan = message.parentSpan ? tracing.PubsubSpans.createPublishFlowSpan(message) : undefined; const doPublish = () => { diff --git a/src/publisher/index.ts b/src/publisher/index.ts index 6ffd8538e..c685af977 100644 --- a/src/publisher/index.ts +++ b/src/publisher/index.ts @@ -339,8 +339,8 @@ export class Publisher { return undefined; } - if (message.telemetrySpan) { - return message.telemetrySpan; + if (message.parentSpan) { + return message.parentSpan; } const span = tracing.PubsubSpans.createPublisherSpan( diff --git a/src/publisher/message-queues.ts b/src/publisher/message-queues.ts index 9b14e3938..fb40a58b3 100644 --- a/src/publisher/message-queues.ts +++ b/src/publisher/message-queues.ts @@ -104,23 +104,20 @@ export abstract class MessageQueue extends EventEmitter { // Make sure we have a projectId filled in to update telemetry spans. // The overall spans may not have the correct projectId because it wasn't // known at the time publishMessage was called. - const spanMessages = messages.filter(m => !!m.telemetrySpan); + const spanMessages = messages.filter(m => !!m.parentSpan); if (spanMessages.length) { if (!topic.pubsub.isIdResolved) { await topic.pubsub.getClientConfig(); } spanMessages.forEach(m => { - tracing.PubsubSpans.updatePublisherTopicName( - m.telemetrySpan!, - topic.name - ); + tracing.PubsubSpans.updatePublisherTopicName(m.parentSpan!, topic.name); }); } messages.forEach(m => { const span = tracing.PubsubSpans.createPublishRpcSpan(m, messages.length); if (span) { - m.telemetryRpc = span; + m.rpcSpan = span; } }); @@ -147,8 +144,8 @@ export abstract class MessageQueue extends EventEmitter { messages.forEach(m => { // We're finished with both the RPC and the whole publish operation, // so close out all of the related spans. - m.telemetryRpc?.end(); - m.telemetrySpan?.end(); + m.rpcSpan?.end(); + m.parentSpan?.end(); }); } } @@ -189,8 +186,7 @@ export class Queue extends MessageQueue { this.publish().catch(() => {}); } - message.telemetryBatching = - tracing.PubsubSpans.createPublishBatchSpan(message); + message.batchingSpan = tracing.PubsubSpans.createPublishBatchSpan(message); this.batch.add(message, callback); @@ -243,7 +239,7 @@ export class Queue extends MessageQueue { delete this.pending; } - messages.forEach(m => m.telemetryBatching?.end()); + messages.forEach(m => m.batchingSpan?.end()); await this._publish(messages, callbacks); if (this.batch.messages.length) { diff --git a/src/publisher/pubsub-message.ts b/src/publisher/pubsub-message.ts index 1d74642f3..d979d6eac 100644 --- a/src/publisher/pubsub-message.ts +++ b/src/publisher/pubsub-message.ts @@ -43,18 +43,18 @@ export interface PubsubMessage // don't get to control what these objects are. They come from grpc. /** - * If telemetry is enabled, track the batch span. + * If tracing is enabled, track the batch span. * * @private */ - telemetryBatching?: tracing.Span; + batchingSpan?: tracing.Span; /** - * If telemetry is enabled, track the RPC send time span. + * If tracing is enabled, track the RPC send time span. * * @private */ - telemetryRpc?: tracing.Span; + rpcSpan?: tracing.Span; } /** diff --git a/src/telemetry-tracing.ts b/src/telemetry-tracing.ts index 9c809477a..1181e8dcc 100644 --- a/src/telemetry-tracing.ts +++ b/src/telemetry-tracing.ts @@ -105,18 +105,19 @@ export function isEnabled( * object, which is one of several possible Message classes. (They're * different for publish and subscribe.) * - * Also we add a telemetrySpan optional member for passing around the - * actual Span object within the client library. + * Also we add a parentSpan optional member for passing around the + * actual Span object within the client library. This can be a publish + * or subscriber span, depending on the context. * * @private */ export interface MessageWithAttributes { attributes?: Attributes | null | undefined; - telemetrySpan?: Span; + parentSpan?: Span; } /** - * Implements the TextMap getter and setter interfaces for Pub/Sub messages. + * Implements common members for the TextMap getter and setter interfaces for Pub/Sub messages. * * @private */ @@ -134,6 +135,11 @@ export class PubsubMessageGetSet { } } +/** + * Implements the TextMap getter interface for Pub/Sub messages. + * + * @private + */ export class PubsubMessageGet extends PubsubMessageGetSet implements TextMapGetter @@ -146,6 +152,11 @@ export class PubsubMessageGet } } +/** + * Implements the TextMap setter interface for Pub/Sub messages. + * + * @private + */ export class PubsubMessageSet extends PubsubMessageGetSet implements TextMapSetter @@ -269,7 +280,7 @@ export class PubsubSpans { message: MessageWithAttributes, name: string ): Span | undefined { - const parent = message.telemetrySpan; + const parent = message.parentSpan; if (parent) { return getTracer().startSpan( name, @@ -406,7 +417,7 @@ export function injectSpan( // Also put the direct reference to the Span object for while we're // passing it around in the client library. - message.telemetrySpan = span; + message.parentSpan = span; } /** @@ -415,7 +426,7 @@ export function injectSpan( * @private */ export function containsSpanContext(message: MessageWithAttributes): boolean { - if (message.telemetrySpan) { + if (message.parentSpan) { return true; } @@ -443,8 +454,8 @@ export function extractSpan( subName: string, enabled: OpenTelemetryLevel ): Span | undefined { - if (message.telemetrySpan) { - return message.telemetrySpan; + if (message.parentSpan) { + return message.parentSpan; } const keys = Object.getOwnPropertyNames(message.attributes ?? {}); @@ -474,7 +485,7 @@ export function extractSpan( } const span = PubsubSpans.createReceiveSpan(message, subName, context); - message.telemetrySpan = span; + message.parentSpan = span; return span; } diff --git a/test/publisher/index.ts b/test/publisher/index.ts index c30373e90..7dc97e3d1 100644 --- a/test/publisher/index.ts +++ b/test/publisher/index.ts @@ -199,7 +199,7 @@ describe('Publisher', () => { // publishMessage is only the first part of the process now, // so we need to manually end the span. - msg.telemetrySpan?.end(); + msg.parentSpan?.end(); const spans = exporter.getFinishedSpans(); assert.notStrictEqual(spans.length, 0, 'has span'); From 5f33931d1d1fd8d988920cb2d373dcbda1bf695c Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Fri, 18 Aug 2023 14:39:20 -0400 Subject: [PATCH 49/52] tests: more renaming issues --- test/lease-manager.ts | 4 ++-- test/subscriber.ts | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/test/lease-manager.ts b/test/lease-manager.ts index 359caa833..9e91f5c96 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -61,7 +61,7 @@ class FakeSubscriberTelemetry { class FakeMessage { length = 20; received: number; - telemetrySub: FakeSubscriberTelemetry = new FakeSubscriberTelemetry(); + subSpans: FakeSubscriberTelemetry = new FakeSubscriberTelemetry(); constructor() { this.received = Date.now(); @@ -71,7 +71,7 @@ class FakeMessage { return AckResponses.Success; } ackFailed() {} - endTelemetrySpan() {} + endParentSpan() {} } interface LeaseManagerInternals { diff --git a/test/subscriber.ts b/test/subscriber.ts index 5a2f87a03..d658eac4a 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -775,10 +775,10 @@ describe('Subscriber', () => { // OTel is enabled during tests, so we need to delete the baggage. // eslint-disable-next-line @typescript-eslint/no-explicit-any const [addMsgAny, msgAny] = [addMsg as any, message as any]; - delete addMsgAny.telemetrySpan; - delete addMsgAny.telemetrySub; - delete msgAny.telemetrySpan; - delete msgAny.telemetrySub; + delete addMsgAny.parentSpan; + delete addMsgAny.subSpans; + delete msgAny.parentSpan; + delete msgAny.subSpans; assert.deepStrictEqual(addMsg, message); From f16788533226819a803abc99d76a526b21df9a52 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Fri, 18 Aug 2023 14:50:28 -0400 Subject: [PATCH 50/52] tests: add otel test for FlowControlledPublisher --- test/publisher/flow-publisher.ts | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/test/publisher/flow-publisher.ts b/test/publisher/flow-publisher.ts index cdeebdb37..55e600917 100644 --- a/test/publisher/flow-publisher.ts +++ b/test/publisher/flow-publisher.ts @@ -27,10 +27,11 @@ import { } from '../../src/publisher'; import {FlowControl} from '../../src/publisher/flow-control'; import * as fp from '../../src/publisher/flow-publisher'; +import * as tracing from '../../src/telemetry-tracing'; class FakePublisher { flowControl!: FlowControl; - publishMessage() {} + async publishMessage() {} setOptions(options: PublishOptions) { this.flowControl.setOptions(options.flowControlOptions!); } @@ -49,6 +50,23 @@ describe('Flow control publisher', () => { sandbox.restore(); }); + it('should create a flow span if a parent exists', async () => { + const fcp = new fp.FlowControlledPublisher(publisher); + const message = { + data: Buffer.from('foo'), + parentSpan: tracing.PubsubSpans.createPublisherSpan({}, 'topic'), + }; + fcp.publish(message as unknown as PubsubMessage); + assert.strictEqual(!!message.parentSpan, true); + }); + + it('should not create a flow span if no parent exists', async () => { + const fcp = new fp.FlowControlledPublisher(publisher); + const message = {data: Buffer.from('foo'), parentSpan: undefined}; + fcp.publish(message as unknown as PubsubMessage); + assert.strictEqual(!message.parentSpan, true); + }); + it('should get no promise if there is flow control space left', async () => { publisher.setOptions({ flowControlOptions: { From feeccf1b1b45180a2758215503df6b508bc74ef1 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Fri, 18 Aug 2023 15:24:31 -0400 Subject: [PATCH 51/52] tests: add lease manager tests for otel --- test/lease-manager.ts | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/test/lease-manager.ts b/test/lease-manager.ts index 9e91f5c96..503129446 100644 --- a/test/lease-manager.ts +++ b/test/lease-manager.ts @@ -146,6 +146,15 @@ describe('LeaseManager', () => { }); describe('add', () => { + it('should start a flow span', () => { + const message = new FakeMessage() as {} as Message; + const stub = sandbox.spy(message.subSpans, 'flowStart'); + + leaseManager.add(message); + + assert.strictEqual(stub.calledOnce, true); + }); + it('should update the bytes/size values', () => { const message = new FakeMessage() as {} as Message; @@ -379,6 +388,14 @@ describe('LeaseManager', () => { assert.strictEqual(leaseManager.size, 0); }); + it('should end all parent spans', () => { + const messages = [new FakeMessage(), new FakeMessage()]; + const spies = messages.map(m => sandbox.spy(m, 'endParentSpan')); + messages.forEach(m => leaseManager.add(m as {} as Message)); + leaseManager.clear(); + spies.forEach(s => assert.strictEqual(s.calledOnce, true)); + }); + it('should emit the free event if it was full', done => { leaseManager.setOptions({maxMessages: 1}); leaseManager.add(new FakeMessage() as {} as Message); @@ -435,6 +452,16 @@ describe('LeaseManager', () => { }); describe('remove', () => { + it('should end the span', () => { + const message = new FakeMessage(); + const spy = sandbox.spy(message, 'endParentSpan'); + + leaseManager.add(message as {} as Message); + leaseManager.remove(message as {} as Message); + + assert.strictEqual(spy.calledOnce, true); + }); + it('should noop for unknown messages', () => { const message = new FakeMessage(); From b218d5ff999eaf9e87c1cb4154f156679560c312 Mon Sep 17 00:00:00 2001 From: feywind <57276408+feywind@users.noreply.github.com> Date: Mon, 21 Aug 2023 15:10:50 -0400 Subject: [PATCH 52/52] tests: add SubscriberSpans unit tests --- src/subscriber.ts | 10 ++--- test/subscriber.ts | 105 +++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 109 insertions(+), 6 deletions(-) diff --git a/src/subscriber.ts b/src/subscriber.ts index 30f5eeb68..cdebdb5c9 100644 --- a/src/subscriber.ts +++ b/src/subscriber.ts @@ -68,13 +68,11 @@ export class AckError extends Error { * @private */ export class SubscriberSpans { - parent: Message; - sub: Subscriber; + parent: tracing.MessageWithAttributes; - // These are always attached to a message (and its subscriber). - constructor(parent: Message, sub: Subscriber) { + // These are always attached to a message. + constructor(parent: tracing.MessageWithAttributes) { this.parent = parent; - this.sub = sub; } // Start a flow control span if needed. @@ -309,7 +307,7 @@ export class Message implements tracing.MessageWithAttributes { * * @private */ - this.subSpans = new SubscriberSpans(this, sub); + this.subSpans = new SubscriberSpans(this); this._handled = false; this._length = this.data.length; diff --git a/test/subscriber.ts b/test/subscriber.ts index d658eac4a..8ef946699 100644 --- a/test/subscriber.ts +++ b/test/subscriber.ts @@ -35,6 +35,7 @@ import * as s from '../src/subscriber'; import {Subscription} from '../src/subscription'; import {SpanKind} from '@opentelemetry/api'; import {Duration} from '../src'; +import * as tracing from '../src/telemetry-tracing'; type PullResponse = google.pubsub.v1.IStreamingPullResponse; @@ -1171,4 +1172,108 @@ describe('Subscriber', () => { }); }); }); + + describe('SubscriberSpans', () => { + const message: tracing.MessageWithAttributes = { + attributes: {}, + parentSpan: undefined, + }; + const spans = new s.SubscriberSpans(message); + const fakeSpan = { + end() {}, + } as unknown as opentelemetry.Span; + + it('starts a flow span', () => { + const stub = sandbox + .stub(tracing.PubsubSpans, 'createReceiveFlowSpan') + .returns(fakeSpan); + spans.flowStart(); + assert.strictEqual(stub.calledOnce, true); + assert.strictEqual(stub.args[0][0], message); + spans.flowStart(); + assert.strictEqual(stub.calledOnce, true); + }); + + it('ends a flow span', () => { + sandbox + .stub(tracing.PubsubSpans, 'createReceiveFlowSpan') + .returns(fakeSpan); + spans.flowStart(); + const spy = sandbox.spy(fakeSpan, 'end'); + spans.flowEnd(); + assert.strictEqual(spy.calledOnce, true); + spans.flowEnd(); + assert.strictEqual(spy.calledOnce, true); + }); + + it('starts a modAck span', () => { + const stub = sandbox + .stub(tracing.PubsubSpans, 'createModAckSpan') + .returns(fakeSpan); + spans.modAckStart(Duration.from({seconds: 10}), true); + assert.strictEqual(stub.args[0][0], message); + assert.strictEqual(stub.args[0][1].totalOf('second'), 10); + assert.strictEqual(stub.args[0][2], true); + spans.modAckStart(Duration.from({seconds: 20}), false); + assert.strictEqual(stub.calledOnce, true); + }); + + it('ends a modAck span', () => { + sandbox.stub(tracing.PubsubSpans, 'createModAckSpan').returns(fakeSpan); + spans.modAckStart(Duration.from({seconds: 10}), true); + const spy = sandbox.spy(fakeSpan, 'end'); + spans.modAckStop(); + assert.strictEqual(spy.calledOnce, true); + spans.modAckStop(); + assert.strictEqual(spy.calledOnce, true); + }); + + it('starts a scheduler span', () => { + const stub = sandbox + .stub(tracing.PubsubSpans, 'createReceiveSchedulerSpan') + .returns(fakeSpan); + spans.schedulerStart(); + assert.strictEqual(stub.args[0][0], message); + assert.strictEqual(stub.calledOnce, true); + spans.schedulerStart(); + assert.strictEqual(stub.calledOnce, true); + }); + + it('ends a scheduler span', () => { + sandbox + .stub(tracing.PubsubSpans, 'createReceiveSchedulerSpan') + .returns(fakeSpan); + spans.schedulerStart(); + const spy = sandbox.spy(fakeSpan, 'end'); + spans.schedulerEnd(); + assert.strictEqual(spy.calledOnce, true); + spans.schedulerEnd(); + assert.strictEqual(spy.calledOnce, true); + }); + + it('starts a processing span', () => { + const stub = sandbox + .stub(tracing.PubsubSpans, 'createReceiveProcessSpan') + .returns(fakeSpan); + const subName = 'foozle'; + spans.processingStart(subName); + assert.strictEqual(stub.args[0][0], message); + assert.strictEqual(stub.args[0][1], subName); + assert.strictEqual(stub.calledOnce, true); + spans.processingStart('boo'); + assert.strictEqual(stub.calledOnce, true); + }); + + it('ends a processing span', () => { + sandbox + .stub(tracing.PubsubSpans, 'createReceiveSchedulerSpan') + .returns(fakeSpan); + spans.processingStart('foozle'); + const spy = sandbox.spy(fakeSpan, 'end'); + spans.processingEnd(); + assert.strictEqual(spy.calledOnce, true); + spans.processingEnd(); + assert.strictEqual(spy.calledOnce, true); + }); + }); });