From 7927a87df517818dac2302936d0b8282fe09eb67 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Mon, 9 Dec 2024 16:07:29 -0800 Subject: [PATCH 01/39] add args type parameter to StoredProcedure --- src/dbos-executor.ts | 14 +++++++------- src/dbos.ts | 4 ++-- src/debugger/debug_workflow.ts | 4 ++-- src/httpServer/handler.ts | 4 ++-- src/procedure.ts | 3 +-- src/testing/testing_runtime.ts | 2 +- src/workflow.ts | 12 ++++++------ 7 files changed, 21 insertions(+), 22 deletions(-) diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index 4b6591fa9..fcfe19866 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -99,7 +99,7 @@ interface StepRegInfo { } interface ProcedureRegInfo { - procedure: StoredProcedure; + procedure: StoredProcedure; config: StoredProcedureConfig; registration: MethodRegistrationBase; } @@ -453,7 +453,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - async callProcedure(proc: StoredProcedure, args: unknown[]): Promise { + async callProcedure(proc: StoredProcedure, args: unknown[]): Promise { const client = await this.procedurePool.connect(); const log = (msg: NoticeMessage) => this.#logNotice(msg); @@ -551,7 +551,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } #registerProcedure(ro: MethodRegistrationBase) { - const proc = ro.registeredFunction as StoredProcedure; + const proc = ro.registeredFunction as StoredProcedure; const cfn = ro.className + '.' + ro.name; if (this.procedureInfoMap.has(cfn)) { @@ -639,11 +639,11 @@ export class DBOSExecutor implements DBOSExecutorContext { return {commInfo, clsInst: getConfiguredInstance(className, cfgName)}; } - getProcedureClassName(pf: StoredProcedure) { + getProcedureClassName(pf: StoredProcedure) { return getRegisteredMethodClassName(pf); } - getProcedureInfo(pf: StoredProcedure) { + getProcedureInfo(pf: StoredProcedure) { const pfName = getRegisteredMethodClassName(pf) + '.' + pf.name; return this.procedureInfoMap.get(pfName); } @@ -994,11 +994,11 @@ export class DBOSExecutor implements DBOSExecutorContext { } - async procedure(proc: StoredProcedure, params: WorkflowParams, ...args: unknown[]): Promise { + async procedure(proc: StoredProcedure, params: WorkflowParams, ...args: T): Promise { // Create a workflow and call procedure. const temp_workflow = async (ctxt: WorkflowContext, ...args: unknown[]) => { const ctxtImpl = ctxt as WorkflowContextImpl; - return await ctxtImpl.procedure(proc, ...args); + return await ctxtImpl.procedure(proc, ...(args as T)); }; return (await this.workflow(temp_workflow, { ...params, diff --git a/src/dbos.ts b/src/dbos.ts index a13ad7c63..8c16ace81 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -51,7 +51,7 @@ import { WorkflowHandle } from "."; import { ConfiguredInstance } from "."; -import { StoredProcedureFunc } from "./procedure"; +import { StoredProcedure } from "./procedure"; import { APITypes } from "./httpServer/handlerTypes"; import { HandlerRegistrationBase } from "./httpServer/handler"; @@ -549,7 +549,7 @@ export class DBOS { ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callStepFunction( op.registeredFunction as StepFunction, null, wfctx, ...args) : op.procConfig - ? (...args: unknown[]) => wfctx.procedure(op.registeredFunction as StoredProcedureFunc, ...args) + ? (...args: unknown[]) => wfctx.procedure(op.registeredFunction as StoredProcedure, ...args) : undefined; } return proxy as InvokeFuncs; diff --git a/src/debugger/debug_workflow.ts b/src/debugger/debug_workflow.ts index a2d3a0ae3..4593d3e9e 100644 --- a/src/debugger/debug_workflow.ts +++ b/src/debugger/debug_workflow.ts @@ -70,7 +70,7 @@ export class WorkflowContextDebug extends DBOSContextImpl implements WorkflowCon : op.commConfig ? (...args: unknown[]) => this.external(op.registeredFunction as StepFunction, null, ...args) : op.procConfig - ? (...args: unknown[]) => this.procedure(op.registeredFunction as StoredProcedure, ...args) + ? (...args: unknown[]) => this.procedure(op.registeredFunction as StoredProcedure, ...args) : undefined; } return proxy as WFInvokeFuncs; @@ -207,7 +207,7 @@ export class WorkflowContextDebug extends DBOSContextImpl implements WorkflowCon return check.output; // Always return the recorded result. } - async procedure(proc: StoredProcedure, ...args: unknown[]): Promise { + async procedure(proc: StoredProcedure, ...args: T): Promise { const procInfo = this.#dbosExec.getProcedureInfo(proc); if (procInfo === undefined) { throw new DBOSDebuggerError(proc.name); } const funcId = this.functionIDGetIncrement(); diff --git a/src/httpServer/handler.ts b/src/httpServer/handler.ts index 8c698f5d6..dca14cd26 100644 --- a/src/httpServer/handler.ts +++ b/src/httpServer/handler.ts @@ -155,7 +155,7 @@ export class HandlerContextImpl extends DBOSContextImpl implements HandlerContex ? (...args: unknown[]) => this.#external(op.registeredFunction as StepFunction, params, ...args) : op.procConfig - ? (...args: unknown[]) => this.#procedure(op.registeredFunction as StoredProcedure, params, ...args) + ? (...args: unknown[]) => this.#procedure(op.registeredFunction as StoredProcedure, params, ...args) : undefined; } else { @@ -218,7 +218,7 @@ export class HandlerContextImpl extends DBOSContextImpl implements HandlerContex return this.#dbosExec.external(stepFn, params, ...args); } - async #procedure(proc: StoredProcedure, params: WorkflowParams, ...args: unknown[]): Promise { + async #procedure(proc: StoredProcedure, params: WorkflowParams, ...args: T): Promise { return this.#dbosExec.procedure(proc, params, ...args); } } diff --git a/src/procedure.ts b/src/procedure.ts index 5393c0cf0..1ccc51075 100644 --- a/src/procedure.ts +++ b/src/procedure.ts @@ -23,8 +23,7 @@ export interface QueryResult extends QueryResultBase { rows: R[]; } -export type StoredProcedure = (ctxt: StoredProcedureContext, ...args: unknown[]) => Promise; -export type StoredProcedureFunc = StoredProcedure; +export type StoredProcedure = (ctxt: StoredProcedureContext, ...args: T) => Promise; export interface StoredProcedureContext extends Pick { query(sql: string, ...params: unknown[]): Promise>; diff --git a/src/testing/testing_runtime.ts b/src/testing/testing_runtime.ts index 3f704600d..53802c0f4 100644 --- a/src/testing/testing_runtime.ts +++ b/src/testing/testing_runtime.ts @@ -207,7 +207,7 @@ export class TestingRuntimeImpl implements TestingRuntime { : op.commConfig ? (...args: unknown[]) => dbosExec.external(op.registeredFunction as StepFunction, wfParams, ...args) : op.procConfig - ? (...args: unknown[]) => dbosExec.procedure(op.registeredFunction as StoredProcedure, wfParams, ...args) + ? (...args: unknown[]) => dbosExec.procedure(op.registeredFunction as StoredProcedure, wfParams, ...args) : undefined; } else { proxy[op.name] = op.workflowConfig diff --git a/src/workflow.ts b/src/workflow.ts index d8b81ae41..ba886caec 100644 --- a/src/workflow.ts +++ b/src/workflow.ts @@ -454,7 +454,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont return this.startChildWorkflow(wf, ...args); } - async #invokeProcLocal(proc: StoredProcedure, args: unknown[], span: Span, config: StoredProcedureConfig, funcId: number) { + async #invokeProcLocal(proc: StoredProcedure, args: T, span: Span, config: StoredProcedureConfig, funcId: number) { let retryWaitMillis = 1; const backoffFactor = 1.5; const maxRetryWaitMs = 2000; // Maximum wait 2 seconds. @@ -540,7 +540,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont } } - async #invokeProcRemote(proc: StoredProcedure, args: unknown[], span: Span, config: StoredProcedureConfig, funcId: number) { + async #invokeProcRemote(proc: StoredProcedure, args: T, span: Span, config: StoredProcedureConfig, funcId: number) { const readOnly = config.readOnly ?? false; const $jsonCtx = { @@ -552,7 +552,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont // Note, node-pg converts JS arrays to postgres array literals, so must call JSON.strigify on // args and bufferedResults before being passed to dbosExec.callProcedure - const $args = [this.workflowUUID, funcId, this.presetUUID, $jsonCtx, null, JSON.stringify(args)]; + const $args = [this.workflowUUID, funcId, this.presetUUID, $jsonCtx, null, JSON.stringify(args)] as unknown[]; if (!readOnly) { // function_id, output, txn_snapshot, created_at const bufferedResults = new Array<[number, unknown, string, number?]>(); @@ -565,7 +565,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont } type ReturnValue = { return_value: { output?: R, error?: unknown, txn_id?: string, txn_snapshot?: string, created_at?: number } }; - const [{ return_value }] = await this.#dbosExec.callProcedure(proc, $args); + const [{ return_value }] = await this.#dbosExec.callProcedure(proc as StoredProcedure, $args); const { error, output, txn_snapshot, txn_id, created_at } = return_value; @@ -601,7 +601,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont } // TODO: ConfiguredInstance support - async procedure(proc: StoredProcedure, ...args: unknown[]): Promise { + async procedure(proc: StoredProcedure, ...args: T): Promise { const procInfo = this.#dbosExec.getProcedureInfo(proc); if (procInfo === undefined) { throw new DBOSNotRegisteredError(proc.name); } const executeLocally = procInfo.config.executeLocally ?? false; @@ -717,7 +717,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont : op.commConfig ? (...args: unknown[]) => this.external(op.registeredFunction as StepFunction, null, ...args) : op.procConfig - ? (...args: unknown[]) => this.procedure(op.registeredFunction as StoredProcedure, ...args) + ? (...args: unknown[]) => this.procedure(op.registeredFunction as StoredProcedure, ...args) : undefined; } return proxy as WFInvokeFuncs; From 29fc08b4d224b6a162f67dce862e7da24c849698 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Tue, 10 Dec 2024 14:32:59 -0800 Subject: [PATCH 02/39] tests passing --- src/dbos-executor.ts | 365 +++++++++++++++++++++++++++------ src/debugger/debug_workflow.ts | 2 +- src/workflow.ts | 242 +--------------------- 3 files changed, 303 insertions(+), 306 deletions(-) diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index fcfe19866..d65e2d807 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -41,21 +41,22 @@ import { import { MethodRegistrationBase, getRegisteredOperations, getOrCreateClassRegistration, MethodRegistration, getRegisteredMethodClassName, getRegisteredMethodName, getConfiguredInstance, ConfiguredInstance, getAllRegisteredClasses } from './decorators'; import { SpanStatusCode } from '@opentelemetry/api'; import knex, { Knex } from 'knex'; -import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext } from './context'; +import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext, runWithDBOSContext } from './context'; import { HandlerRegistrationBase } from './httpServer/handler'; import { WorkflowContextDebug } from './debugger/debug_workflow'; -import { serializeError } from 'serialize-error'; +import { deserializeError, serializeError } from 'serialize-error'; import { DBOSJSON, sleepms } from './utils'; import path from 'node:path'; -import { StoredProcedure, StoredProcedureConfig } from './procedure'; +import { StoredProcedure, StoredProcedureConfig, StoredProcedureContextImpl } from './procedure'; import { NoticeMessage } from "pg-protocol/dist/messages"; -import { DBOSEventReceiver, DBOSExecutorContext, GetWorkflowsInput, GetWorkflowsOutput} from "."; +import { DBOSEventReceiver, DBOSExecutorContext, GetWorkflowsInput, GetWorkflowsOutput } from "."; import { get } from "lodash"; import { wfQueueRunner, WorkflowQueue } from "./wfqueue"; import { debugTriggerPoint, DEBUG_TRIGGER_WORKFLOW_ENQUEUE } from "./debugpoint"; import { DBOSScheduler } from './scheduler/scheduler'; import { DBOSEventReceiverState, DBOSEventReceiverQuery, DBNotificationCallback, DBNotificationListener } from "./eventreceiver"; +import { transaction_outputs } from "../schemas/user_db_schema"; // eslint-disable-next-line @typescript-eslint/no-empty-object-type @@ -125,6 +126,8 @@ const TempWorkflowType = { send: "send", } as const; +type QueryFunction = (sql: string, args: unknown[]) => Promise; + export class DBOSExecutor implements DBOSExecutorContext { initialized: boolean; // User Database @@ -302,7 +305,7 @@ export class DBOSExecutor implements DBOSExecutorContext { const DrizzleExports = require("drizzle-orm/node-postgres"); const drizzlePool = new Pool(userDBConfig); // eslint-disable-next-line @typescript-eslint/no-unsafe-assignment, @typescript-eslint/no-unsafe-call, @typescript-eslint/no-unsafe-member-access - const drizzle= DrizzleExports.drizzle(drizzlePool, {schema: this.drizzleEntities}); + const drizzle = DrizzleExports.drizzle(drizzlePool, { schema: this.drizzleEntities }); // eslint-disable-next-line @typescript-eslint/no-unsafe-argument this.userDatabase = new DrizzleUserDatabase(drizzlePool, drizzle); this.logger.debug("Loaded Drizzle user database"); @@ -332,12 +335,12 @@ export class DBOSExecutor implements DBOSExecutorContext { } getRegistrationsFor(obj: DBOSEventReceiver) { - const res: {methodConfig: unknown, classConfig: unknown, methodReg: MethodRegistrationBase}[] = []; + const res: { methodConfig: unknown, classConfig: unknown, methodReg: MethodRegistrationBase }[] = []; for (const r of this.registeredOperations) { if (!r.eventReceiverInfo.has(obj)) continue; const methodConfig = r.eventReceiverInfo.get(obj)!; const classConfig = r.defaults?.eventReceiverInfo.get(obj) ?? {}; - res.push({methodReg: r, methodConfig, classConfig}) + res.push({ methodReg: r, methodConfig, classConfig }) } return res; } @@ -453,26 +456,6 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - async callProcedure(proc: StoredProcedure, args: unknown[]): Promise { - const client = await this.procedurePool.connect(); - const log = (msg: NoticeMessage) => this.#logNotice(msg); - - const procClassName = this.getProcedureClassName(proc); - const plainProcName = `${procClassName}_${proc.name}_p`; - const procName = this.config.appVersion - ? `v${this.config.appVersion}_${plainProcName}` - : plainProcName; - - const sql = `CALL "${procName}"(${args.map((_v, i) => `$${i + 1}`).join()});`; - try { - client.on('notice', log); - return await client.query(sql, args).then(value => value.rows); - } finally { - client.off('notice', log); - client.release(); - } - } - async destroy() { if (this.pendingWorkflowMap.size > 0) { this.logger.info("Waiting for pending workflows to finish."); @@ -501,7 +484,7 @@ export class DBOSExecutor implements DBOSExecutorContext { /* WORKFLOW OPERATIONS */ - #registerWorkflow(ro :MethodRegistrationBase) { + #registerWorkflow(ro: MethodRegistrationBase) { const wf = ro.registeredFunction as Workflow; if (wf.name === DBOSExecutor.tempWorkflowName) { throw new DBOSError(`Unexpected use of reserved workflow name: ${wf.name}`); @@ -512,7 +495,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } const workflowInfo: WorkflowRegInfo = { workflow: wf, - config: {...ro.workflowConfig}, + config: { ...ro.workflowConfig }, registration: ro, }; this.workflowInfoMap.set(wfn, workflowInfo); @@ -528,7 +511,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } const txnInfo: TransactionRegInfo = { transaction: txf, - config: {...ro.txnConfig}, + config: { ...ro.txnConfig }, registration: ro, }; this.transactionInfoMap.set(tfn, txnInfo); @@ -543,7 +526,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } const stepInfo: StepRegInfo = { step: comm, - config: {...ro.commConfig}, + config: { ...ro.commConfig }, registration: ro, }; this.stepInfoMap.set(cfn, stepInfo); @@ -559,7 +542,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } const procInfo: ProcedureRegInfo = { procedure: proc, - config: {...ro.procConfig}, + config: { ...ro.procConfig }, registration: ro, }; this.procedureInfoMap.set(cfn, procInfo); @@ -588,7 +571,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - return {wfInfo, configuredInst: getConfiguredInstance(wf.workflowClassName, wf.workflowConfigName)}; + return { wfInfo, configuredInst: getConfiguredInstance(wf.workflowClassName, wf.workflowConfigName) }; } getTransactionInfo(tf: Transaction) { @@ -612,7 +595,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - return {txnInfo, clsInst: getConfiguredInstance(className, cfgName)}; + return { txnInfo, clsInst: getConfiguredInstance(className, cfgName) }; } getStepInfo(cf: StepFunction) { @@ -636,7 +619,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - return {commInfo, clsInst: getConfiguredInstance(className, cfgName)}; + return { commInfo, clsInst: getConfiguredInstance(className, cfgName) }; } getProcedureClassName(pf: StoredProcedure) { @@ -700,7 +683,7 @@ export class DBOSExecutor implements DBOSExecutorContext { // Synchronously set the workflow's status to PENDING and record workflow inputs (for non single-transaction workflows). // We have to do it for all types of workflows because operation_outputs table has a foreign key constraint on workflow status table. if ((wCtxt.tempWfOperationType !== TempWorkflowType.transaction - && wCtxt.tempWfOperationType !== TempWorkflowType.procedure) + && wCtxt.tempWfOperationType !== TempWorkflowType.procedure) || params.queueName !== undefined ) { // TODO: Make this transactional (and with the queue step below) @@ -714,7 +697,7 @@ export class DBOSExecutor implements DBOSExecutorContext { // Execute the workflow. try { let cresult: R | undefined; - await runWithWorkflowContext(wCtxt, async ()=> { + await runWithWorkflowContext(wCtxt, async () => { if (passContext) { cresult = await wf.call(params.configuredInstance, wCtxt, ...args); } @@ -743,7 +726,7 @@ export class DBOSExecutor implements DBOSExecutorContext { wCtxt.span.setStatus({ code: SpanStatusCode.OK }); } else { // Record the error. - const e = err as Error & {dbos_already_logged?: boolean}; + const e = err as Error & { dbos_already_logged?: boolean }; this.logger.error(e); e.dbos_already_logged = true if (wCtxt.isTempWorkflow) { @@ -847,10 +830,54 @@ export class DBOSExecutor implements DBOSExecutorContext { } return recordedResult; // Always return the recorded result. }); - }); + }); return new InvokedHandle(this.systemDatabase, workflowPromise, workflowUUID, wf.name, callerUUID, callerFunctionID); } + /** + * Retrieve the transaction snapshot information of the current transaction + */ + static async #retrieveSnapshot(query: QueryFunction): Promise { + const rows = await query<{ txn_snapshot: string }>("SELECT pg_current_snapshot()::text as txn_snapshot;", []); + return rows[0].txn_snapshot; + } + + /** + * Check if an operation has already executed in a workflow. + * If it previously executed successfully, return its output. + * If it previously executed and threw an error, throw that error. + * Otherwise, return DBOSNull. + * Also return the transaction snapshot information of this current transaction. + */ + async #checkExecution(query: QueryFunction, workflowUUID: string, funcID: number): Promise { + // Note: we read the current snapshot, not the recorded one! + const rows = await query( + "(SELECT output, error, txn_snapshot, true as recorded FROM dbos.transaction_outputs WHERE workflow_uuid=$1 AND function_id=$2 UNION ALL SELECT null as output, null as error, pg_current_snapshot()::text as txn_snapshot, false as recorded) ORDER BY recorded", + [workflowUUID, funcID] + ); + + if (rows.length === 0 || rows.length > 2) { + this.logger.error("Unexpected! This should never happen. Returned rows: " + rows.toString()); + throw new DBOSError("This should never happen. Returned rows: " + rows.toString()); + } + + const res: BufferedResult = { + output: dbosNull, + txn_snapshot: "" + } + // recorded=false row will be first because we used ORDER BY. + res.txn_snapshot = rows[0].txn_snapshot; + if (rows.length === 2) { + if (DBOSJSON.parse(rows[1].error) !== null) { + throw deserializeError(DBOSJSON.parse(rows[1].error)); + } else { + res.output = DBOSJSON.parse(rows[1].output) as R; + } + } + return res; + } + + async transaction(txn: Transaction, params: WorkflowParams, ...args: T): Promise { // Create a workflow and call transaction. const temp_workflow = async (ctxt: WorkflowContext, ...args: T) => { @@ -902,7 +929,8 @@ export class DBOSExecutor implements DBOSExecutorContext { // If the UUID is preset, it is possible this execution previously happened. Check, and return its original result if it did. // Note: It is possible to retrieve a generated ID from a workflow handle, run a concurrent execution, and cause trouble for yourself. We recommend against this. if (wfCtx.presetUUID) { - const check: BufferedResult = await wfCtx.checkTxExecution(client, funcId); + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + const check: BufferedResult = await this.#checkExecution(func, workflowUUID, funcId); txn_snapshot = check.txn_snapshot; if (check.output !== dbosNull) { tCtxt.span.setAttribute("cached", true); @@ -912,7 +940,8 @@ export class DBOSExecutor implements DBOSExecutorContext { } } else { // Collect snapshot information for read-only transactions and non-preset UUID transactions, if not already collected above - txn_snapshot = await wfCtx.retrieveTxSnapshot(client); + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + txn_snapshot = await DBOSExecutor.#retrieveSnapshot(func); } // For non-read-only transactions, flush the result buffer. @@ -923,13 +952,13 @@ export class DBOSExecutor implements DBOSExecutorContext { // Execute the user's transaction. let cresult: R | undefined; if (txnInfo.registration.passContext) { - await runWithTransactionContext(tCtxt, async ()=> { + await runWithTransactionContext(tCtxt, async () => { cresult = await txn.call(clsinst, tCtxt, ...args); }); } else { - await runWithTransactionContext(tCtxt, async ()=> { - const tf = txn as unknown as (...args: T)=>Promise; + await runWithTransactionContext(tCtxt, async () => { + const tf = txn as unknown as (...args: T) => Promise; cresult = await tf.call(clsinst, ...args); }); } @@ -993,22 +1022,230 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - async procedure(proc: StoredProcedure, params: WorkflowParams, ...args: T): Promise { // Create a workflow and call procedure. - const temp_workflow = async (ctxt: WorkflowContext, ...args: unknown[]) => { + const temp_workflow = async (ctxt: WorkflowContext, ...args: T) => { const ctxtImpl = ctxt as WorkflowContextImpl; - return await ctxtImpl.procedure(proc, ...(args as T)); + return this.callProcedureFunction(proc, ctxtImpl, ...args); }; - return (await this.workflow(temp_workflow, - { ...params, + return await (await this.workflow(temp_workflow, + { + ...params, tempWfType: TempWorkflowType.procedure, tempWfName: getRegisteredMethodName(proc), tempWfClass: getRegisteredMethodClassName(proc), }, ...args)).getResult(); } - async executeProcedure(func: (client: PoolClient) => Promise, config: TransactionConfig): Promise { + async callProcedureFunction( + proc: StoredProcedure, wfCtx: WorkflowContextImpl, ...args: T + ): Promise { + const procInfo = this.getProcedureInfo(proc); + if (procInfo === undefined) { + throw new DBOSNotRegisteredError(proc.name); + } + + const executeLocally = procInfo.config.executeLocally ?? false; + const funcId = wfCtx.functionIDGetIncrement(); + const span: Span = this.tracer.startSpan( + proc.name, + { + operationUUID: wfCtx.workflowUUID, + operationType: OperationType.PROCEDURE, + authenticatedUser: wfCtx.authenticatedUser, + assumedRole: wfCtx.assumedRole, + authenticatedRoles: wfCtx.authenticatedRoles, + readOnly: procInfo.config.readOnly ?? false, + isolationLevel: procInfo.config.isolationLevel, + executeLocally, + }, + wfCtx.span, + ); + + try { + const result = executeLocally + ? await this.#callProcedureFunctionLocal(proc, args, wfCtx, span, procInfo.config, funcId) + : await this.#callProcedureFunctionRemote(proc, args, wfCtx, span, procInfo.config, funcId); + span.setStatus({ code: SpanStatusCode.OK }); + return result; + } catch (e) { + const { message } = e as { message: string }; + span.setStatus({ code: SpanStatusCode.ERROR, message }); + throw e; + } finally { + this.tracer.endSpan(span); + } + } + + async #callProcedureFunctionLocal(proc: StoredProcedure, args: T, wfCtx: WorkflowContextImpl, span: Span, config: StoredProcedureConfig, funcId: number): Promise { + let retryWaitMillis = 1; + const backoffFactor = 1.5; + const maxRetryWaitMs = 2000; // Maximum wait 2 seconds. + const readOnly = config.readOnly ?? false; + + while (true) { + let txn_snapshot = "invalid"; + const wrappedProcedure = async (client: PoolClient): Promise => { + const ctxt = new StoredProcedureContextImpl(client, wfCtx, span, this.logger, proc.name); + + if (wfCtx.presetUUID) { + const func = (sql: string, args: unknown[]) => this.procedurePool.query(sql, args).then(v => v.rows as T[]); + const check: BufferedResult = await this.#checkExecution(func, wfCtx.workflowUUID, wfCtx.functionID); + txn_snapshot = check.txn_snapshot; + if (check.output !== dbosNull) { + ctxt.span.setAttribute("cached", true); + ctxt.span.setStatus({ code: SpanStatusCode.OK }); + this.tracer.endSpan(ctxt.span); + return check.output as R; + } + } else { + // Collect snapshot information for read-only transactions and non-preset UUID transactions, if not already collected above + const func = (sql: string, args: unknown[]) => this.procedurePool.query(sql, args).then(v => v.rows as T[]); + txn_snapshot = await DBOSExecutor.#retrieveSnapshot(func); + } + + // For non-read-only transactions, flush the result buffer. + if (!readOnly) { + await wfCtx.flushResultBufferProc(client); + } + + let cresult: R | undefined; + await runWithDBOSContext(ctxt, async () => { + cresult = await proc(ctxt, ...args); + }); + const result = cresult! + + if (readOnly) { + // Buffer the output of read-only transactions instead of synchronously writing it. + const readOutput: BufferedResult = { + output: result, + txn_snapshot: txn_snapshot, + created_at: Date.now(), + } + wfCtx.resultBuffer.set(funcId, readOutput); + } else { + // Synchronously record the output of write transactions and obtain the transaction ID. + const pg_txn_id = await wfCtx.recordOutputProc(client, funcId, txn_snapshot, result); + ctxt.span.setAttribute("pg_txn_id", pg_txn_id); + wfCtx.resultBuffer.clear(); + } + + return result; + }; + + try { + const result = await this.invokeStoredProcFunction(wrappedProcedure, { isolationLevel: config.isolationLevel }); + span.setStatus({ code: SpanStatusCode.OK }); + return result; + } catch (err) { + if (this.userDatabase.isRetriableTransactionError(err)) { + // serialization_failure in PostgreSQL + span.addEvent("TXN SERIALIZATION FAILURE", { "retryWaitMillis": retryWaitMillis }, performance.now()); + // Retry serialization failures. + await sleepms(retryWaitMillis); + retryWaitMillis *= backoffFactor; + retryWaitMillis = retryWaitMillis < maxRetryWaitMs ? retryWaitMillis : maxRetryWaitMs; + continue; + } + + // Record and throw other errors. + const e: Error = err as Error; + await this.invokeStoredProcFunction(async (client: PoolClient) => { + await wfCtx.flushResultBufferProc(client); + await wfCtx.recordErrorProc(client, funcId, txn_snapshot, e); + }, { isolationLevel: IsolationLevel.ReadCommitted }); + + await this.userDatabase.transaction(async (client: UserDatabaseClient) => { + await wfCtx.flushResultBuffer(client); + await wfCtx.recordErrorTx(client, funcId, txn_snapshot, e); + }, { isolationLevel: IsolationLevel.ReadCommitted }); + wfCtx.resultBuffer.clear(); + throw err; + } + } + } + + async #callProcedureFunctionRemote(proc: StoredProcedure, args: T, wfCtx: WorkflowContextImpl, span: Span, config: StoredProcedureConfig, funcId: number): Promise { + const readOnly = config.readOnly ?? false; + + const $jsonCtx = { + request: wfCtx.request, + authenticatedUser: wfCtx.authenticatedUser, + authenticatedRoles: wfCtx.authenticatedRoles, + assumedRole: wfCtx.assumedRole, + }; + + // Note, node-pg converts JS arrays to postgres array literals, so must call JSON.strigify on + // args and bufferedResults before being passed to #invokeStoredProc + const $args = [wfCtx.workflowUUID, funcId, wfCtx.presetUUID, $jsonCtx, null, JSON.stringify(args)] as unknown[]; + if (!readOnly) { + // function_id, output, txn_snapshot, created_at + const bufferedResults = new Array<[number, unknown, string, number?]>(); + for (const [functionID, { output, txn_snapshot, created_at }] of wfCtx.resultBuffer.entries()) { + bufferedResults.push([functionID, output, txn_snapshot, created_at]); + } + // sort by function ID + bufferedResults.sort((a, b) => a[0] - b[0]); + $args.unshift(bufferedResults.length > 0 ? JSON.stringify(bufferedResults) : null); + } + + type ReturnValue = { return_value: { output?: R, error?: unknown, txn_id?: string, txn_snapshot?: string, created_at?: number } }; + const [{ return_value }] = await this.#invokeStoredProc(proc as StoredProcedure, $args); + + const { error, output, txn_snapshot, txn_id, created_at } = return_value; + + // buffered results are persisted in r/w stored procs, even if it returns an error + if (!readOnly) { + wfCtx.resultBuffer.clear(); + } + + // if the stored proc returns an error, deserialize and throw it. + // stored proc saves the error in tx_output before returning + if (error) { + throw deserializeError(error); + } + + // if txn_snapshot is provided, the output needs to be buffered + if (readOnly && txn_snapshot) { + wfCtx.resultBuffer.set(funcId, { + output, + txn_snapshot, + created_at: created_at ?? Date.now(), + }); + } + + if (!readOnly) { + wfCtx.resultBuffer.clear(); + } + + if (txn_id) { + span.setAttribute("pg_txn_id", txn_id); + } + span.setStatus({ code: SpanStatusCode.OK }); + return output!; + } + + async #invokeStoredProc(proc: StoredProcedure, args: unknown[]): Promise { + const client = await this.procedurePool.connect(); + const log = (msg: NoticeMessage) => this.#logNotice(msg); + + const procClassName = this.getProcedureClassName(proc); + const plainProcName = `${procClassName}_${proc.name}_p`; + const procName = this.config.appVersion + ? `v${this.config.appVersion}_${plainProcName}` + : plainProcName; + + const sql = `CALL "${procName}"(${args.map((_v, i) => `$${i + 1}`).join()});`; + try { + client.on('notice', log); + return await client.query(sql, args).then(value => value.rows); + } finally { + client.off('notice', log); + client.release(); + } + } + + async invokeStoredProcFunction(func: (client: PoolClient) => Promise, config: TransactionConfig): Promise { const client = await this.procedurePool.connect(); try { const readOnly = config.readOnly ?? false; @@ -1104,12 +1341,12 @@ export class DBOSExecutor implements DBOSExecutorContext { try { let cresult: R | undefined; if (commInfo.registration.passContext) { - await runWithStepContext(ctxt, async ()=> { + await runWithStepContext(ctxt, async () => { cresult = await stepFn.call(clsInst, ctxt, ...args); }); } else { - await runWithStepContext(ctxt, async ()=> { + await runWithStepContext(ctxt, async () => { const sf = stepFn as unknown as (...args: T) => Promise; cresult = await sf.call(clsInst, ...args); }); @@ -1132,12 +1369,12 @@ export class DBOSExecutor implements DBOSExecutorContext { try { let cresult: R | undefined; if (commInfo.registration.passContext) { - await runWithStepContext(ctxt, async ()=> { + await runWithStepContext(ctxt, async () => { cresult = await stepFn.call(clsInst, ctxt, ...args); }); } else { - await runWithStepContext(ctxt, async ()=> { + await runWithStepContext(ctxt, async () => { const sf = stepFn as unknown as (...args: T) => Promise; cresult = await sf.call(clsInst, ...args); }); @@ -1223,10 +1460,10 @@ export class DBOSExecutor implements DBOSExecutorContext { close: async () => { for (const nname of channels) { try { - await notificationsClient.query(`UNLISTEN ${nname};`); + await notificationsClient.query(`UNLISTEN ${nname};`); } - catch(e) { - this.logger.warn(e); + catch (e) { + this.logger.warn(e); } notificationsClient.release(); } @@ -1285,7 +1522,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } const parentCtx = this.#getRecoveryContext(workflowUUID, wfStatus); - const {wfInfo, configuredInst} = this.getWorkflowInfoByStatus(wfStatus); + const { wfInfo, configuredInst } = this.getWorkflowInfoByStatus(wfStatus); // If starting a new workflow, assign a new UUID. Otherwise, use the workflow's original UUID. const workflowStartUUID = startNewWorkflow ? undefined : workflowUUID; @@ -1295,8 +1532,8 @@ export class DBOSExecutor implements DBOSExecutorContext { workflowUUID: workflowStartUUID, parentCtx: parentCtx, configuredInstance: configuredInst, recovery: true, queueName: wfStatus.queueName, executeWorkflow: true, }, - // eslint-disable-next-line @typescript-eslint/no-unsafe-argument - ...inputs); + // eslint-disable-next-line @typescript-eslint/no-unsafe-argument + ...inputs); } // Should be temporary workflows. Parse the name of the workflow. @@ -1313,7 +1550,7 @@ export class DBOSExecutor implements DBOSExecutorContext { let tempWfName: string | undefined; let tempWfClass: string | undefined; if (nameArr[1] === TempWorkflowType.transaction) { - const {txnInfo, clsInst} = this.getTransactionInfoByNames(wfStatus.workflowClassName, nameArr[2], wfStatus.workflowConfigName); + const { txnInfo, clsInst } = this.getTransactionInfoByNames(wfStatus.workflowClassName, nameArr[2], wfStatus.workflowConfigName); if (!txnInfo) { this.logger.error(`Cannot find transaction info for UUID ${workflowUUID}, name ${nameArr[2]}`); throw new DBOSNotRegisteredError(nameArr[2]); @@ -1327,7 +1564,7 @@ export class DBOSExecutor implements DBOSExecutorContext { }; clsinst = clsInst; } else if (nameArr[1] === TempWorkflowType.external) { - const {commInfo, clsInst} = this.getStepInfoByNames(wfStatus.workflowClassName, nameArr[2], wfStatus.workflowConfigName); + const { commInfo, clsInst } = this.getStepInfoByNames(wfStatus.workflowClassName, nameArr[2], wfStatus.workflowConfigName); if (!commInfo) { this.logger.error(`Cannot find step info for UUID ${workflowUUID}, name ${nameArr[2]}`); throw new DBOSNotRegisteredError(nameArr[2]); @@ -1355,8 +1592,8 @@ export class DBOSExecutor implements DBOSExecutorContext { workflowUUID: workflowStartUUID, parentCtx: parentCtx ?? undefined, configuredInstance: clsinst, recovery: true, tempWfType, tempWfClass, tempWfName, }, - // eslint-disable-next-line @typescript-eslint/no-unsafe-argument - ...inputs); + // eslint-disable-next-line @typescript-eslint/no-unsafe-argument + ...inputs); } async getEventDispatchState(svc: string, wfn: string, key: string): Promise { diff --git a/src/debugger/debug_workflow.ts b/src/debugger/debug_workflow.ts index 4593d3e9e..a7f14d7f4 100644 --- a/src/debugger/debug_workflow.ts +++ b/src/debugger/debug_workflow.ts @@ -250,7 +250,7 @@ export class WorkflowContextDebug extends DBOSContextImpl implements WorkflowCon let result: Awaited | Error; try { - result = await this.#dbosExec.executeProcedure(wrappedProcedure, procInfo.config); + result = await this.#dbosExec.invokeStoredProcFunction(wrappedProcedure, procInfo.config); } catch (e) { result = e as Error; } diff --git a/src/workflow.ts b/src/workflow.ts index ba886caec..e7525ca4a 100644 --- a/src/workflow.ts +++ b/src/workflow.ts @@ -215,69 +215,6 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont return this.functionID++; } - /** - * Retrieve the transaction snapshot information of the current transaction - */ - async #retrieveSnapshot(query: QueryFunction): Promise { - const rows = await query<{ txn_snapshot: string }>("SELECT pg_current_snapshot()::text as txn_snapshot;", []); - return rows[0].txn_snapshot; - } - - retrieveTxSnapshot(client: UserDatabaseClient): Promise { - const func = (sql: string, args: unknown[]) => this.#dbosExec.userDatabase.queryWithClient(client, sql, ...args); - return this.#retrieveSnapshot(func); - } - - retrieveProcSnapshot(client: PoolClient): Promise { - const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); - return this.#retrieveSnapshot(func); - } - - /** - * Check if an operation has already executed in a workflow. - * If it previously executed successfully, return its output. - * If it previously executed and threw an error, throw that error. - * Otherwise, return DBOSNull. - * Also return the transaction snapshot information of this current transaction. - */ - async #checkExecution(query: QueryFunction, funcID: number): Promise { - // Note: we read the current snapshot, not the recorded one! - const rows = await query( - "(SELECT output, error, txn_snapshot, true as recorded FROM dbos.transaction_outputs WHERE workflow_uuid=$1 AND function_id=$2 UNION ALL SELECT null as output, null as error, pg_current_snapshot()::text as txn_snapshot, false as recorded) ORDER BY recorded", - [this.workflowUUID, funcID] - ); - - if (rows.length === 0 || rows.length > 2) { - this.logger.error("Unexpected! This should never happen. Returned rows: " + rows.toString()); - throw new DBOSError("This should never happen. Returned rows: " + rows.toString()); - } - - const res: BufferedResult = { - output: dbosNull, - txn_snapshot: "" - } - // recorded=false row will be first because we used ORDER BY. - res.txn_snapshot = rows[0].txn_snapshot; - if (rows.length === 2) { - if (DBOSJSON.parse(rows[1].error) !== null) { - throw deserializeError(DBOSJSON.parse(rows[1].error)); - } else { - res.output = DBOSJSON.parse(rows[1].output) as R; - } - } - return res; - } - - checkTxExecution(client: UserDatabaseClient, funcID: number): Promise { - const func = (sql: string, args: unknown[]) => this.#dbosExec.userDatabase.queryWithClient(client, sql, ...args); - return this.#checkExecution(func, funcID); - } - - checkProcExecution(client: PoolClient, funcID: number): Promise { - const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); - return this.#checkExecution(func, funcID); - } - /** * Write all entries in the workflow result buffer to the database. * If it encounters a primary key error, this indicates a concurrent execution with the same UUID, so throw an DBOSError. @@ -454,186 +391,9 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont return this.startChildWorkflow(wf, ...args); } - async #invokeProcLocal(proc: StoredProcedure, args: T, span: Span, config: StoredProcedureConfig, funcId: number) { - let retryWaitMillis = 1; - const backoffFactor = 1.5; - const maxRetryWaitMs = 2000; // Maximum wait 2 seconds. - const readOnly = config.readOnly ?? false; - - while (true) { - let txn_snapshot = "invalid"; - const wrappedProcedure = async (client: PoolClient): Promise => { - const ctxt = new StoredProcedureContextImpl(client, this, span, this.#dbosExec.logger, proc.name); - - if (this.presetUUID) { - const check: BufferedResult = await this.checkProcExecution(client, this.functionID); - txn_snapshot = check.txn_snapshot; - if (check.output !== dbosNull) { - ctxt.span.setAttribute("cached", true); - ctxt.span.setStatus({ code: SpanStatusCode.OK }); - this.#dbosExec.tracer.endSpan(ctxt.span); - return check.output as R; - } - } else { - // Collect snapshot information for read-only transactions and non-preset UUID transactions, if not already collected above - txn_snapshot = await this.retrieveProcSnapshot(client); - } - - // For non-read-only transactions, flush the result buffer. - if (!readOnly) { - await this.flushResultBufferProc(client); - } - - let cresult: R | undefined; - await runWithDBOSContext(ctxt, async ()=> { - cresult = await proc(ctxt, ...args); - }); - const result = cresult! - - if (readOnly) { - // Buffer the output of read-only transactions instead of synchronously writing it. - const readOutput: BufferedResult = { - output: result, - txn_snapshot: txn_snapshot, - created_at: Date.now(), - } - this.resultBuffer.set(funcId, readOutput); - } else { - // Synchronously record the output of write transactions and obtain the transaction ID. - const pg_txn_id = await this.recordOutputProc(client, funcId, txn_snapshot, result); - ctxt.span.setAttribute("pg_txn_id", pg_txn_id); - this.resultBuffer.clear(); - } - - return result; - }; - - try { - const result = await this.#dbosExec.executeProcedure(wrappedProcedure, { isolationLevel: config.isolationLevel }); - span.setStatus({ code: SpanStatusCode.OK }); - return result; - } catch (err) { - if (this.#dbosExec.userDatabase.isRetriableTransactionError(err)) { - // serialization_failure in PostgreSQL - span.addEvent("TXN SERIALIZATION FAILURE", { "retryWaitMillis": retryWaitMillis }, performance.now()); - // Retry serialization failures. - await sleepms(retryWaitMillis); - retryWaitMillis *= backoffFactor; - retryWaitMillis = retryWaitMillis < maxRetryWaitMs ? retryWaitMillis : maxRetryWaitMs; - continue; - } - - // Record and throw other errors. - const e: Error = err as Error; - await this.#dbosExec.executeProcedure(async (client: PoolClient) => { - await this.flushResultBufferProc(client); - await this.recordErrorProc(client, funcId, txn_snapshot, e); - }, { isolationLevel: IsolationLevel.ReadCommitted }); - - await this.#dbosExec.userDatabase.transaction(async (client: UserDatabaseClient) => { - await this.flushResultBuffer(client); - await this.recordErrorTx(client, funcId, txn_snapshot, e); - }, { isolationLevel: IsolationLevel.ReadCommitted }); - this.resultBuffer.clear(); - throw err; - } - } - } - - async #invokeProcRemote(proc: StoredProcedure, args: T, span: Span, config: StoredProcedureConfig, funcId: number) { - const readOnly = config.readOnly ?? false; - - const $jsonCtx = { - request: this.request, - authenticatedUser: this.authenticatedUser, - authenticatedRoles: this.authenticatedRoles, - assumedRole: this.assumedRole, - }; - - // Note, node-pg converts JS arrays to postgres array literals, so must call JSON.strigify on - // args and bufferedResults before being passed to dbosExec.callProcedure - const $args = [this.workflowUUID, funcId, this.presetUUID, $jsonCtx, null, JSON.stringify(args)] as unknown[]; - if (!readOnly) { - // function_id, output, txn_snapshot, created_at - const bufferedResults = new Array<[number, unknown, string, number?]>(); - for (const [functionID, { output, txn_snapshot, created_at }] of this.resultBuffer.entries()) { - bufferedResults.push([functionID, output, txn_snapshot, created_at]); - } - // sort by function ID - bufferedResults.sort((a, b) => a[0] - b[0]); - $args.unshift(bufferedResults.length > 0 ? JSON.stringify(bufferedResults) : null); - } - - type ReturnValue = { return_value: { output?: R, error?: unknown, txn_id?: string, txn_snapshot?: string, created_at?: number } }; - const [{ return_value }] = await this.#dbosExec.callProcedure(proc as StoredProcedure, $args); - - const { error, output, txn_snapshot, txn_id, created_at } = return_value; - - // buffered results are persisted in r/w stored procs, even if it returns an error - if (!readOnly) { - this.resultBuffer.clear(); - } - - // if the stored proc returns an error, deserialize and throw it. - // stored proc saves the error in tx_output before returning - if (error) { - throw deserializeError(error); - } - - // if txn_snapshot is provided, the output needs to be buffered - if (readOnly && txn_snapshot) { - this.resultBuffer.set(funcId, { - output, - txn_snapshot, - created_at: created_at ?? Date.now(), - }); - } - - if (!readOnly) { - this.resultBuffer.clear(); - } - - if (txn_id) { - span.setAttribute("pg_txn_id", txn_id); - } - span.setStatus({ code: SpanStatusCode.OK }); - return output!; - } - // TODO: ConfiguredInstance support async procedure(proc: StoredProcedure, ...args: T): Promise { - const procInfo = this.#dbosExec.getProcedureInfo(proc); - if (procInfo === undefined) { throw new DBOSNotRegisteredError(proc.name); } - const executeLocally = procInfo.config.executeLocally ?? false; - const funcId = this.functionIDGetIncrement(); - const span: Span = this.#dbosExec.tracer.startSpan( - proc.name, - { - operationUUID: this.workflowUUID, - operationType: OperationType.PROCEDURE, - authenticatedUser: this.authenticatedUser, - assumedRole: this.assumedRole, - authenticatedRoles: this.authenticatedRoles, - readOnly: procInfo.config.readOnly ?? false, - isolationLevel: procInfo.config.isolationLevel, - executeLocally, - }, - this.span, - ); - - try { - const result = executeLocally - ? await this.#invokeProcLocal(proc, args, span, procInfo.config, funcId) - : await this.#invokeProcRemote(proc, args, span, procInfo.config, funcId); - span.setStatus({ code: SpanStatusCode.OK }); - return result; - } catch (e) { - const { message } = e as { message: string }; - span.setStatus({ code: SpanStatusCode.ERROR, message }); - throw e; - } finally { - this.#dbosExec.tracer.endSpan(span); - } + return this.#dbosExec.callProcedureFunction(proc, this, ...args); } /** From dfacda8f865ec6ccd1bc0c813bbe1aa2436bbe2c Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Tue, 10 Dec 2024 16:54:31 -0800 Subject: [PATCH 03/39] more logic move from WF ctx -> Dbos Exec --- src/dbos-executor.ts | 131 ++++++++++++++++++++++++++++++++++++------- src/workflow.ts | 116 ++------------------------------------ 2 files changed, 116 insertions(+), 131 deletions(-) diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index d65e2d807..d1f08fb7d 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -37,6 +37,7 @@ import { KnexUserDatabase, DrizzleUserDatabase, UserDatabaseClient, + pgNodeIsKeyConflictError, } from './user_database'; import { MethodRegistrationBase, getRegisteredOperations, getOrCreateClassRegistration, MethodRegistration, getRegisteredMethodClassName, getRegisteredMethodName, getConfiguredInstance, ConfiguredInstance, getAllRegisteredClasses } from './decorators'; import { SpanStatusCode } from '@opentelemetry/api'; @@ -835,20 +836,20 @@ export class DBOSExecutor implements DBOSExecutorContext { } /** - * Retrieve the transaction snapshot information of the current transaction - */ + * Retrieve the transaction snapshot information of the current transaction + */ static async #retrieveSnapshot(query: QueryFunction): Promise { const rows = await query<{ txn_snapshot: string }>("SELECT pg_current_snapshot()::text as txn_snapshot;", []); return rows[0].txn_snapshot; } /** - * Check if an operation has already executed in a workflow. - * If it previously executed successfully, return its output. - * If it previously executed and threw an error, throw that error. - * Otherwise, return DBOSNull. - * Also return the transaction snapshot information of this current transaction. - */ + * Check if an operation has already executed in a workflow. + * If it previously executed successfully, return its output. + * If it previously executed and threw an error, throw that error. + * Otherwise, return DBOSNull. + * Also return the transaction snapshot information of this current transaction. + */ async #checkExecution(query: QueryFunction, workflowUUID: string, funcID: number): Promise { // Note: we read the current snapshot, not the recorded one! const rows = await query( @@ -877,6 +878,91 @@ export class DBOSExecutor implements DBOSExecutorContext { return res; } + /** + * Write a operation's output to the database. + */ + static async #recordOutput(query: QueryFunction, workflowUUID: string, funcID: number, txnSnapshot: string, output: R, isKeyConflict: (error: unknown) => boolean): Promise { + try { + const serialOutput = DBOSJSON.stringify(output); + const rows = await query("INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, output, txn_id, txn_snapshot, created_at) VALUES ($1, $2, $3, (select pg_current_xact_id_if_assigned()::text), $4, $5) RETURNING txn_id;", + [workflowUUID, funcID, serialOutput, txnSnapshot, Date.now()]); + return rows[0].txn_id; + } catch (error) { + if (isKeyConflict(error)) { + // Serialization and primary key conflict (Postgres). + throw new DBOSWorkflowConflictUUIDError(workflowUUID); + } else { + throw error; + } + } + } + + /** + * Record an error in an operation to the database. + */ + static async #recordError(query: QueryFunction, workflowUUID: string, funcID: number, txnSnapshot: string, err: Error, isKeyConflict: (error: unknown) => boolean): Promise { + try { + const serialErr = DBOSJSON.stringify(serializeError(err)); + await query("INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, error, txn_id, txn_snapshot, created_at) VALUES ($1, $2, $3, null, $4, $5) RETURNING txn_id;", + [workflowUUID, funcID, serialErr, txnSnapshot, Date.now()]); + } catch (error) { + if (isKeyConflict(error)) { + // Serialization and primary key conflict (Postgres). + throw new DBOSWorkflowConflictUUIDError(workflowUUID); + } else { + throw error; + } + } + } + + /** + * Write all entries in the workflow result buffer to the database. + * If it encounters a primary key error, this indicates a concurrent execution with the same UUID, so throw an DBOSError. + */ + async #flushResultBuffer(query: QueryFunction, resultBuffer: Map, workflowUUID: string, isKeyConflict: (error: unknown) => boolean): Promise { + const funcIDs = Array.from(resultBuffer.keys()); + if (funcIDs.length === 0) { + return; + } + funcIDs.sort(); + try { + let sqlStmt = "INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, output, error, txn_id, txn_snapshot, created_at) VALUES "; + let paramCnt = 1; + const values: any[] = []; + for (const funcID of funcIDs) { + // Capture output and also transaction snapshot information. + // Initially, no txn_id because no queries executed. + const recorded = resultBuffer.get(funcID); + const output = recorded!.output; + const txnSnapshot = recorded!.txn_snapshot; + const createdAt = recorded!.created_at!; + if (paramCnt > 1) { + sqlStmt += ", "; + } + sqlStmt += `($${paramCnt++}, $${paramCnt++}, $${paramCnt++}, $${paramCnt++}, null, $${paramCnt++}, $${paramCnt++})`; + values.push(workflowUUID, funcID, DBOSJSON.stringify(output), DBOSJSON.stringify(null), txnSnapshot, createdAt); + } + this.logger.debug(sqlStmt); + await query(sqlStmt, values); + } catch (error) { + if (isKeyConflict(error)) { + // Serialization and primary key conflict (Postgres). + throw new DBOSWorkflowConflictUUIDError(workflowUUID); + } else { + throw error; + } + } + } + + flushResultBuffer(client: UserDatabaseClient, resultBuffer: Map, workflowUUID: string): Promise { + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + return this.#flushResultBuffer(func, resultBuffer, workflowUUID, (error) => this.userDatabase.isKeyConflictError(error)); + } + + #flushResultBufferProc(client: PoolClient, resultBuffer: Map, workflowUUID: string): Promise { + const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); + return this.#flushResultBuffer(func, resultBuffer, workflowUUID, pgNodeIsKeyConflictError); + } async transaction(txn: Transaction, params: WorkflowParams, ...args: T): Promise { // Create a workflow and call transaction. @@ -946,7 +1032,7 @@ export class DBOSExecutor implements DBOSExecutorContext { // For non-read-only transactions, flush the result buffer. if (!readOnly) { - await wfCtx.flushResultBuffer(client); + await this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); } // Execute the user's transaction. @@ -976,7 +1062,8 @@ export class DBOSExecutor implements DBOSExecutorContext { } else { try { // Synchronously record the output of write transactions and obtain the transaction ID. - const pg_txn_id = await wfCtx.recordOutputTx(client, funcId, txn_snapshot, result); + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + const pg_txn_id = await DBOSExecutor.#recordOutput(func, wfCtx.workflowUUID, funcId, txn_snapshot, result, (error) => this.userDatabase.isKeyConflictError(error)); tCtxt.span.setAttribute("pg_txn_id", pg_txn_id); wfCtx.resultBuffer.clear(); } catch (error) { @@ -1011,8 +1098,9 @@ export class DBOSExecutor implements DBOSExecutorContext { // Record and throw other errors. const e: Error = err as Error; await this.userDatabase.transaction(async (client: UserDatabaseClient) => { - await wfCtx.flushResultBuffer(client); - await wfCtx.recordErrorTx(client, funcId, txn_snapshot, e); + await this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + await DBOSExecutor.#recordError(func, wfCtx.workflowUUID, funcId, txn_snapshot, e, (error) => this.userDatabase.isKeyConflictError(error)); }, { isolationLevel: IsolationLevel.ReadCommitted }); wfCtx.resultBuffer.clear(); span.setStatus({ code: SpanStatusCode.ERROR, message: e.message }); @@ -1106,7 +1194,7 @@ export class DBOSExecutor implements DBOSExecutorContext { // For non-read-only transactions, flush the result buffer. if (!readOnly) { - await wfCtx.flushResultBufferProc(client); + await this.#flushResultBufferProc(client, wfCtx.resultBuffer, wfCtx.workflowUUID); } let cresult: R | undefined; @@ -1125,7 +1213,10 @@ export class DBOSExecutor implements DBOSExecutorContext { wfCtx.resultBuffer.set(funcId, readOutput); } else { // Synchronously record the output of write transactions and obtain the transaction ID. - const pg_txn_id = await wfCtx.recordOutputProc(client, funcId, txn_snapshot, result); + const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); + const pg_txn_id = await DBOSExecutor.#recordOutput(func, wfCtx.workflowUUID, funcId, txn_snapshot, result, pgNodeIsKeyConflictError); + + // const pg_txn_id = await wfCtx.recordOutputProc(client, funcId, txn_snapshot, result); ctxt.span.setAttribute("pg_txn_id", pg_txn_id); wfCtx.resultBuffer.clear(); } @@ -1151,13 +1242,15 @@ export class DBOSExecutor implements DBOSExecutorContext { // Record and throw other errors. const e: Error = err as Error; await this.invokeStoredProcFunction(async (client: PoolClient) => { - await wfCtx.flushResultBufferProc(client); - await wfCtx.recordErrorProc(client, funcId, txn_snapshot, e); + await this.#flushResultBufferProc(client, wfCtx.resultBuffer, wfCtx.workflowUUID); + const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); + await DBOSExecutor.#recordError(func, wfCtx.workflowUUID, funcId, txn_snapshot, e, pgNodeIsKeyConflictError); }, { isolationLevel: IsolationLevel.ReadCommitted }); await this.userDatabase.transaction(async (client: UserDatabaseClient) => { - await wfCtx.flushResultBuffer(client); - await wfCtx.recordErrorTx(client, funcId, txn_snapshot, e); + this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); + const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); + await DBOSExecutor.#recordError(func, wfCtx.workflowUUID, funcId, txn_snapshot, e, (error) => this.userDatabase.isKeyConflictError(error)); }, { isolationLevel: IsolationLevel.ReadCommitted }); wfCtx.resultBuffer.clear(); throw err; @@ -1314,7 +1407,7 @@ export class DBOSExecutor implements DBOSExecutorContext { const ctxt: StepContextImpl = new StepContextImpl(wfCtx, funcID, span, this.logger, commInfo.config, stepFn.name); await this.userDatabase.transaction(async (client: UserDatabaseClient) => { - await wfCtx.flushResultBuffer(client); + await this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); }, { isolationLevel: IsolationLevel.ReadCommitted }); wfCtx.resultBuffer.clear(); diff --git a/src/workflow.ts b/src/workflow.ts index e7525ca4a..8a9b40e3f 100644 --- a/src/workflow.ts +++ b/src/workflow.ts @@ -172,8 +172,6 @@ export interface WorkflowContext extends DBOSContext { sleep(durationSec: number): Promise; } -type QueryFunction = (sql: string, args: unknown[]) => Promise; - export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowContext { functionID: number = 0; readonly #dbosExec; @@ -215,113 +213,6 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont return this.functionID++; } - /** - * Write all entries in the workflow result buffer to the database. - * If it encounters a primary key error, this indicates a concurrent execution with the same UUID, so throw an DBOSError. - */ - async #flushResultBuffer(query: QueryFunction, isKeyConflict: (error: unknown) => boolean): Promise { - const funcIDs = Array.from(this.resultBuffer.keys()); - if (funcIDs.length === 0) { - return; - } - funcIDs.sort(); - try { - let sqlStmt = "INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, output, error, txn_id, txn_snapshot, created_at) VALUES "; - let paramCnt = 1; - const values: any[] = []; - for (const funcID of funcIDs) { - // Capture output and also transaction snapshot information. - // Initially, no txn_id because no queries executed. - const recorded = this.resultBuffer.get(funcID); - const output = recorded!.output; - const txnSnapshot = recorded!.txn_snapshot; - const createdAt = recorded!.created_at!; - if (paramCnt > 1) { - sqlStmt += ", "; - } - sqlStmt += `($${paramCnt++}, $${paramCnt++}, $${paramCnt++}, $${paramCnt++}, null, $${paramCnt++}, $${paramCnt++})`; - values.push(this.workflowUUID, funcID, DBOSJSON.stringify(output), DBOSJSON.stringify(null), txnSnapshot, createdAt); - } - this.logger.debug(sqlStmt); - await query(sqlStmt, values); - } catch (error) { - if (isKeyConflict(error)) { - // Serialization and primary key conflict (Postgres). - throw new DBOSWorkflowConflictUUIDError(this.workflowUUID); - } else { - throw error; - } - } - } - - flushResultBuffer(client: UserDatabaseClient): Promise { - const func = (sql: string, args: unknown[]) => this.#dbosExec.userDatabase.queryWithClient(client, sql, ...args); - return this.#flushResultBuffer(func, (error) => this.#dbosExec.userDatabase.isKeyConflictError(error)); - } - - flushResultBufferProc(client: PoolClient): Promise { - const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); - return this.#flushResultBuffer(func, pgNodeIsKeyConflictError); - } - - - /** - * Write a operation's output to the database. - */ - async #recordOutput(query: QueryFunction, funcID: number, txnSnapshot: string, output: R, isKeyConflict: (error: unknown) => boolean): Promise { - try { - const serialOutput = DBOSJSON.stringify(output); - const rows = await query("INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, output, txn_id, txn_snapshot, created_at) VALUES ($1, $2, $3, (select pg_current_xact_id_if_assigned()::text), $4, $5) RETURNING txn_id;", - [this.workflowUUID, funcID, serialOutput, txnSnapshot, Date.now()]); - return rows[0].txn_id; - } catch (error) { - if (isKeyConflict(error)) { - // Serialization and primary key conflict (Postgres). - throw new DBOSWorkflowConflictUUIDError(this.workflowUUID); - } else { - throw error; - } - } - } - - recordOutputTx(client: UserDatabaseClient, funcID: number, txnSnapshot: string, output: R): Promise { - const func = (sql: string, args: unknown[]) => this.#dbosExec.userDatabase.queryWithClient(client, sql, ...args); - return this.#recordOutput(func, funcID, txnSnapshot, output, (error) => this.#dbosExec.userDatabase.isKeyConflictError(error)); - } - - recordOutputProc(client: PoolClient, funcID: number, txnSnapshot: string, output: R): Promise { - const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); - return this.#recordOutput(func, funcID, txnSnapshot, output, pgNodeIsKeyConflictError); - } - - /** - * Record an error in an operation to the database. - */ - async #recordError(query: QueryFunction, funcID: number, txnSnapshot: string, err: Error, isKeyConflict: (error: unknown) => boolean): Promise { - try { - const serialErr = DBOSJSON.stringify(serializeError(err)); - await query("INSERT INTO dbos.transaction_outputs (workflow_uuid, function_id, error, txn_id, txn_snapshot, created_at) VALUES ($1, $2, $3, null, $4, $5) RETURNING txn_id;", - [this.workflowUUID, funcID, serialErr, txnSnapshot, Date.now()]); - } catch (error) { - if (isKeyConflict(error)) { - // Serialization and primary key conflict (Postgres). - throw new DBOSWorkflowConflictUUIDError(this.workflowUUID); - } else { - throw error; - } - } - } - - recordErrorTx(client: UserDatabaseClient, funcID: number, txnSnapshot: string, err: Error): Promise { - const func = (sql: string, args: unknown[]) => this.#dbosExec.userDatabase.queryWithClient(client, sql, ...args); - return this.#recordError(func, funcID, txnSnapshot, err, (error) => this.#dbosExec.userDatabase.isKeyConflictError(error)); - } - - recordErrorProc(client: PoolClient, funcID: number, txnSnapshot: string, err: Error): Promise { - const func = (sql: string, args: unknown[]) => client.query(sql, args).then(v => v.rows as T[]); - return this.#recordError(func, funcID, txnSnapshot, err, pgNodeIsKeyConflictError); - } - /** * Invoke another workflow as its child workflow and return a workflow handle. * The child workflow is guaranteed to be executed exactly once, even if the workflow is retried with the same UUID. @@ -378,6 +269,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont return this.proxyInvokeWF(target, workflowUUID, true, target as ConfiguredInstance, queue) as unknown as WfInvokeWfsAsync; } } + invokeWorkflow(target: T, workflowUUID?: string): WfInvokeWfs { if (typeof target === 'function') { return this.proxyInvokeWF(target, workflowUUID, false, null) as unknown as WfInvokeWfs; @@ -423,7 +315,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont const functionID: number = this.functionIDGetIncrement(); await this.#dbosExec.userDatabase.transaction(async (client: UserDatabaseClient) => { - await this.flushResultBuffer(client); + await this.#dbosExec.flushResultBuffer(client, this.resultBuffer, this.workflowUUID); }, { isolationLevel: IsolationLevel.ReadCommitted }); this.resultBuffer.clear(); @@ -440,7 +332,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont const timeoutFunctionID: number = this.functionIDGetIncrement(); await this.#dbosExec.userDatabase.transaction(async (client: UserDatabaseClient) => { - await this.flushResultBuffer(client); + await this.#dbosExec.flushResultBuffer(client, this.resultBuffer, this.workflowUUID); }, { isolationLevel: IsolationLevel.ReadCommitted }); this.resultBuffer.clear(); @@ -455,7 +347,7 @@ export class WorkflowContextImpl extends DBOSContextImpl implements WorkflowCont const functionID: number = this.functionIDGetIncrement(); await this.#dbosExec.userDatabase.transaction(async (client: UserDatabaseClient) => { - await this.flushResultBuffer(client); + await this.#dbosExec.flushResultBuffer(client, this.resultBuffer, this.workflowUUID); }, { isolationLevel: IsolationLevel.ReadCommitted }); this.resultBuffer.clear(); From 3f058d053b3930ffd39388e9fd6bec8fec2c6e06 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Tue, 10 Dec 2024 17:07:36 -0800 Subject: [PATCH 04/39] add procedure to DBOSExecutorContext --- src/eventreceiver.ts | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/eventreceiver.ts b/src/eventreceiver.ts index e64ae5688..1062adde1 100644 --- a/src/eventreceiver.ts +++ b/src/eventreceiver.ts @@ -5,6 +5,7 @@ import { TransactionFunction } from './transaction'; import { MethodRegistrationBase } from './decorators'; import { StepFunction } from './step'; import { Notification } from "pg"; +import { StoredProcedure } from './procedure'; export type DBNotification = Notification; export type DBNotificationCallback = (n: DBNotification) => void; @@ -44,6 +45,8 @@ export interface DBOSExecutorContext transaction(txn: TransactionFunction, params: WorkflowParams, ...args: T): Promise; workflow(wf: WorkflowFunction, params: WorkflowParams, ...args: T): Promise>; external(stepFn: StepFunction, params: WorkflowParams, ...args: T): Promise; + procedure(proc: StoredProcedure, params: WorkflowParams, ...args: T): Promise; + send(destinationUUID: string, message: T, topic?: string, idempotencyKey?: string): Promise; getEvent(workflowUUID: string, key: string, timeoutSeconds?: number): Promise; From 26473cf77e8abd30481efddbd8bae742dbed9635 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 11 Dec 2024 11:02:16 -0800 Subject: [PATCH 05/39] update dbos.invoke proc path to use callProcedureFunction --- src/dbos.ts | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/dbos.ts b/src/dbos.ts index 731e572d1..6f838cdec 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -536,12 +536,13 @@ export class DBOS { for (const op of ops) { proxy[op.name] = op.txnConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callTransactionFunction( - op.registeredFunction as TransactionFunction, null, wfctx, ...args) + op.registeredFunction as TransactionFunction, null, wfctx, ...args) : op.commConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callStepFunction( - op.registeredFunction as StepFunction, null, wfctx, ...args) + op.registeredFunction as StepFunction, null, wfctx, ...args) : op.procConfig - ? (...args: unknown[]) => wfctx.procedure(op.registeredFunction as StoredProcedure, ...args) + ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callProcedureFunction( + op.registeredFunction as StoredProcedure, wfctx, ...args) : undefined; } return proxy as InvokeFuncs; From 0482fbc7dc1dafd0a110a55edda6405f87439a84 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 11 Dec 2024 15:46:45 -0800 Subject: [PATCH 06/39] runWithStoredProcContext --- src/context.ts | 15 +++++++++++++++ src/dbos-executor.ts | 8 ++++---- src/debugger/debug_workflow.ts | 2 +- src/procedure.ts | 1 + 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/context.ts b/src/context.ts index 452148b26..e6261414b 100644 --- a/src/context.ts +++ b/src/context.ts @@ -11,6 +11,7 @@ import { WorkflowContext, WorkflowContextImpl } from "./workflow"; import { TransactionContextImpl } from "./transaction"; import { StepContextImpl } from "./step"; import { DBOSInvalidWorkflowTransitionError } from "./error"; +import { StoredProcedureContextImpl } from "./procedure"; export interface DBOSLocalCtx { ctx?: DBOSContext; @@ -105,6 +106,20 @@ export async function runWithTransactionContext(ctx: StoredProcedureContextImpl, callback: ()=>Promise) { + // Check we are in a workflow context and not in a step / transaction already + const pctx = getCurrentContextStore(); + if (!pctx) throw new DBOSInvalidWorkflowTransitionError(); + if (!isInWorkflowCtx(pctx)) throw new DBOSInvalidWorkflowTransitionError(); + return await asyncLocalCtx.run({ + ctx, + workflowId: ctx.workflowUUID, + curTxFunctionId: ctx.functionID, + parentCtx: pctx, + }, + callback); +} + export async function runWithStepContext(ctx: StepContextImpl, callback: ()=>Promise) { // Check we are in a workflow context and not in a step / transaction already const pctx = getCurrentContextStore(); diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index d1f08fb7d..d12d826e4 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -42,7 +42,7 @@ import { import { MethodRegistrationBase, getRegisteredOperations, getOrCreateClassRegistration, MethodRegistration, getRegisteredMethodClassName, getRegisteredMethodName, getConfiguredInstance, ConfiguredInstance, getAllRegisteredClasses } from './decorators'; import { SpanStatusCode } from '@opentelemetry/api'; import knex, { Knex } from 'knex'; -import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext, runWithDBOSContext } from './context'; +import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext, runWithDBOSContext, runWithStoredProcContext } from './context'; import { HandlerRegistrationBase } from './httpServer/handler'; import { WorkflowContextDebug } from './debugger/debug_workflow'; import { deserializeError, serializeError } from 'serialize-error'; @@ -1174,11 +1174,11 @@ export class DBOSExecutor implements DBOSExecutorContext { while (true) { let txn_snapshot = "invalid"; const wrappedProcedure = async (client: PoolClient): Promise => { - const ctxt = new StoredProcedureContextImpl(client, wfCtx, span, this.logger, proc.name); + const ctxt = new StoredProcedureContextImpl(client, wfCtx, span, this.logger, funcId, proc.name); if (wfCtx.presetUUID) { const func = (sql: string, args: unknown[]) => this.procedurePool.query(sql, args).then(v => v.rows as T[]); - const check: BufferedResult = await this.#checkExecution(func, wfCtx.workflowUUID, wfCtx.functionID); + const check: BufferedResult = await this.#checkExecution(func, wfCtx.workflowUUID, funcId); txn_snapshot = check.txn_snapshot; if (check.output !== dbosNull) { ctxt.span.setAttribute("cached", true); @@ -1198,7 +1198,7 @@ export class DBOSExecutor implements DBOSExecutorContext { } let cresult: R | undefined; - await runWithDBOSContext(ctxt, async () => { + await runWithStoredProcContext(ctxt, async () => { cresult = await proc(ctxt, ...args); }); const result = cresult! diff --git a/src/debugger/debug_workflow.ts b/src/debugger/debug_workflow.ts index a7f14d7f4..6f77fa9f8 100644 --- a/src/debugger/debug_workflow.ts +++ b/src/debugger/debug_workflow.ts @@ -229,7 +229,7 @@ export class WorkflowContextDebug extends DBOSContextImpl implements WorkflowCon let check: RecordedResult | Error; const wrappedProcedure = async (client: PoolClient): Promise => { check = await this.checkProcExecution(client, funcId); - const procCtxt = new StoredProcedureContextImpl(client, this, span, this.#dbosExec.logger, proc.name); + const procCtxt = new StoredProcedureContextImpl(client, this, span, this.#dbosExec.logger, funcId, proc.name); if (check instanceof Error) { if (this.#dbosExec.debugProxy) { diff --git a/src/procedure.ts b/src/procedure.ts index 1ccc51075..7bcba24ef 100644 --- a/src/procedure.ts +++ b/src/procedure.ts @@ -35,6 +35,7 @@ export class StoredProcedureContextImpl extends DBOSContextImpl implements Store workflowContext: WorkflowContextImpl | WorkflowContextDebug, span: Span, logger: Logger, + readonly functionID: number, operationName: string ) { super(operationName, span, logger, workflowContext); From fca2dfc5bf636700a632f8b687cc13b4af4fd89d Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 11 Dec 2024 18:05:46 -0800 Subject: [PATCH 07/39] append random string to proc tests to enable rerun --- tests/proc-test/src/operations.test.ts | 27 +++++++++++++++++++------- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 1ed41bfe4..6b088cf65 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -18,6 +18,19 @@ async function runSql(config: ClientConfig, func: (client: Client) => Promise } } +function randomString(length?: number) { + length ??= 4 + Math.floor(Math.random() * 10); + + let result = ''; + const characters = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; + for (let i = 0; i < length; i++) { + result += characters.charAt(Math.floor(Math.random() * characters.length)); + } + return result; +} + + + describe("operations-test", () => { let config: DBOSConfig; let testRuntime: TestingRuntime; @@ -69,7 +82,7 @@ describe("operations-test", () => { test("test-procGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = "procWF"; + const user = `procWF_${randomString()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -92,7 +105,7 @@ describe("operations-test", () => { test("test-debug-procGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = "debugProcWF"; + const user = `debugProcWF_${randomString()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -112,7 +125,7 @@ describe("operations-test", () => { test("test-txGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = "txWF"; + const user = `txWF_${randomString()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).txAndProcGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -130,7 +143,7 @@ describe("operations-test", () => { test("test-procLocalGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = "procLocalWF"; + const user = `procLocalWF_${randomString()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procLocalGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -153,7 +166,7 @@ describe("operations-test", () => { test("test-txAndProcGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = "txAndProcWF"; + const user = `txAndProcWF_${randomString()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).txAndProcGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -178,7 +191,7 @@ describe("operations-test", () => { try { const wfUUID = uuidv1(); - const user = "txAndProcWFv2"; + const user = `txAndProcWFv2_${randomString()}`; const res = await DBOS.withNextWorkflowID(wfUUID, async () => { return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); }) @@ -203,7 +216,7 @@ describe("operations-test", () => { test("test-procErrorWorkflow", async () => { const wfid = uuidv1(); - const user = "procErrorWF"; + const user = `procErrorWF_${randomString()}`; await expect(testRuntime.invokeWorkflow(StoredProcTest, wfid).procErrorWorkflow(user)).rejects.toThrow("This is a test error"); const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfid); From adf0e705e28839ac97ebf089f26058a750be108c Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 11:33:43 -0800 Subject: [PATCH 08/39] update paths in proc test --- tests/proc-test/package.json | 2 +- tests/proc-test/src/operations.test.ts | 4 ++-- tests/proc-test/tsconfig.json | 1 - 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/proc-test/package.json b/tests/proc-test/package.json index f9a6f89de..b7cefc106 100644 --- a/tests/proc-test/package.json +++ b/tests/proc-test/package.json @@ -2,7 +2,7 @@ "name": "proc-test", "version": "0.0.1", "scripts": { - "build": "tsc", + "build": "tsc -p tsconfig.json", "test": "npx dbos rollback && npx dbos migrate && jest --detectOpenHandles", "lint": "eslint src", "lint-fix": "eslint --fix src", diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 6b088cf65..8c71f9b04 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -2,8 +2,8 @@ import { DBOS, TestingRuntime, parseConfigFile } from "@dbos-inc/dbos-sdk"; import { StoredProcTest } from "./operations"; import { v1 as uuidv1 } from "uuid"; -import { transaction_outputs } from "../../../schemas/user_db_schema"; -import { workflow_status } from "../../../schemas/system_db_schema"; +import { workflow_status } from "@dbos-inc/dbos-sdk/schemas/system_db_schema"; +import { transaction_outputs } from "@dbos-inc/dbos-sdk/schemas/user_db_schema"; import { TestingRuntimeImpl, createInternalTestRuntime } from "@dbos-inc/dbos-sdk/dist/src/testing/testing_runtime"; import { DBOSConfig } from "@dbos-inc/dbos-sdk"; import { Client, ClientConfig } from "pg"; diff --git a/tests/proc-test/tsconfig.json b/tests/proc-test/tsconfig.json index 5488d6185..8bd5845f0 100644 --- a/tests/proc-test/tsconfig.json +++ b/tests/proc-test/tsconfig.json @@ -19,7 +19,6 @@ "src" ], "exclude": [ - "**/*.test.ts", "dist" ] } From 07846a302049f0dde0fddb848e14ea3fa935b099 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 11:36:45 -0800 Subject: [PATCH 09/39] dummy dbos.storedProc --- src/dbos.ts | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/dbos.ts b/src/dbos.ts index 2b8b20138..e364c7c85 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -52,7 +52,7 @@ import { WorkflowHandle } from "."; import { ConfiguredInstance } from "."; -import { StoredProcedure } from "./procedure"; +import { StoredProcedure, StoredProcedureConfig } from "./procedure"; import { APITypes } from "./httpServer/handlerTypes"; import { HandlerRegistrationBase } from "./httpServer/handler"; @@ -815,6 +815,18 @@ export class DBOS { return decorator; } + static storedProcedure(config: StoredProcedureConfig={}) { + function decorator( + target: object, + propertyKey: string, + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) + { + return inDescriptor; + } + + return decorator; + } + static step(config: StepConfig={}) { function decorator( target: object, From dd82ce3daaa533495de46f0777070136e76444a6 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 11:51:05 -0800 Subject: [PATCH 10/39] dbos.storedproc --- src/dbos.ts | 41 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/src/dbos.ts b/src/dbos.ts index e364c7c85..30a32c386 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -821,7 +821,46 @@ export class DBOS { propertyKey: string, inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { - return inDescriptor; + const { descriptor, registration } = registerAndWrapContextFreeFunction(target, propertyKey, inDescriptor); + registration.procConfig = config; + + const invokeWrapper = async function (this: This, ...rawArgs: Args): Promise { + let inst: ConfiguredInstance | undefined = undefined; + if (typeof this === 'function') { + // This is static + } + else { + throw new Error("Stored procedures must be static"); + // inst = this as ConfiguredInstance; + // if (!("name" in inst)) { + // throw new DBOSInvalidWorkflowTransitionError(); + // } + } + + if (DBOS.isWithinWorkflow()) { + const wfctx = assertCurrentWorkflowContext(); + return await DBOSExecutor.globalInstance!.callProcedureFunction( + registration.registeredFunction as unknown as StoredProcedure, wfctx, ...rawArgs); + } + + const wfParams: WorkflowParams = { + // configuredInstance: inst + }; + + return await DBOS.executor.procedure( + registration.registeredFunction as unknown as StoredProcedure, + wfParams, ...rawArgs + ); + }; + + descriptor.value = invokeWrapper; + registration.wrappedFunction = invokeWrapper; + + Object.defineProperty(invokeWrapper, "name", { + value: registration.name, + }); + + return descriptor; } return decorator; From c618af8a7b39933d32ea24eb15d67965299ffad8 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 11:52:00 -0800 Subject: [PATCH 11/39] update proc test --- tests/proc-test/src/operations.ts | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 323ee7d03..3a528f92b 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -126,7 +126,7 @@ export class StoredProcTest { static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; }> { // Retrieve the number of times this user has been greeted. const count = await StoredProcTest.getGreetCountTx_v2(user); - const greeting = await DBOS.invoke(StoredProcTest).helloProcedure(user); + const greeting = await StoredProcTest.helloProcedure_v2(user); return { count, greeting }; } @@ -139,6 +139,14 @@ export class StoredProcTest { if (result && result.rows.length > 0) { return result.rows[0].greet_count; } return 0; } + + @DBOS.storedProcedure() + static async helloProcedure_v2(user: string): Promise { + const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; + const { rows } = await DBOS.pgClient.query(query, [user]); + const greet_count = rows[0].greet_count; + return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; + } } From 6bda3b592ae27886786237931722c820d9abae51 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 16:06:00 -0800 Subject: [PATCH 12/39] use date.now insteasd of random string --- tests/proc-test/src/operations.test.ts | 27 +++++++------------------- 1 file changed, 7 insertions(+), 20 deletions(-) diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 8c71f9b04..84c89801f 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -18,19 +18,6 @@ async function runSql(config: ClientConfig, func: (client: Client) => Promise } } -function randomString(length?: number) { - length ??= 4 + Math.floor(Math.random() * 10); - - let result = ''; - const characters = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; - for (let i = 0; i < length; i++) { - result += characters.charAt(Math.floor(Math.random() * characters.length)); - } - return result; -} - - - describe("operations-test", () => { let config: DBOSConfig; let testRuntime: TestingRuntime; @@ -82,7 +69,7 @@ describe("operations-test", () => { test("test-procGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `procWF_${randomString()}`; + const user = `procWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -105,7 +92,7 @@ describe("operations-test", () => { test("test-debug-procGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `debugProcWF_${randomString()}`; + const user = `debugProcWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -125,7 +112,7 @@ describe("operations-test", () => { test("test-txGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `txWF_${randomString()}`; + const user = `txWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).txAndProcGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -143,7 +130,7 @@ describe("operations-test", () => { test("test-procLocalGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `procLocalWF_${randomString()}`; + const user = `procLocalWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procLocalGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -166,7 +153,7 @@ describe("operations-test", () => { test("test-txAndProcGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `txAndProcWF_${randomString()}`; + const user = `txAndProcWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).txAndProcGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -191,7 +178,7 @@ describe("operations-test", () => { try { const wfUUID = uuidv1(); - const user = `txAndProcWFv2_${randomString()}`; + const user = `txAndProcWFv2_${Date.now()}`; const res = await DBOS.withNextWorkflowID(wfUUID, async () => { return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); }) @@ -216,7 +203,7 @@ describe("operations-test", () => { test("test-procErrorWorkflow", async () => { const wfid = uuidv1(); - const user = `procErrorWF_${randomString()}`; + const user = `procErrorWF_${Date.now()}`; await expect(testRuntime.invokeWorkflow(StoredProcTest, wfid).procErrorWorkflow(user)).rejects.toThrow("This is a test error"); const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfid); From f62cca25726e14eb01eff51353f7b334f831a12c Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 12 Dec 2024 16:06:27 -0800 Subject: [PATCH 13/39] use v1 stored proc in proc test txAndProcGreetingWorkflow_v2 --- tests/proc-test/src/operations.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 3a528f92b..87618e87d 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -126,7 +126,7 @@ export class StoredProcTest { static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; }> { // Retrieve the number of times this user has been greeted. const count = await StoredProcTest.getGreetCountTx_v2(user); - const greeting = await StoredProcTest.helloProcedure_v2(user); + const greeting = await DBOS.invoke(StoredProcTest).helloProcedure(user); return { count, greeting }; } From c6be28d752db18ed61a0bd702f602d55ef5b9e27 Mon Sep 17 00:00:00 2001 From: Chuck B Date: Fri, 13 Dec 2024 21:46:50 -0500 Subject: [PATCH 14/39] Maybe --- tests/proc-test/package.json | 4 ++-- tests/proc-test/tsconfig.json | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/proc-test/package.json b/tests/proc-test/package.json index b7cefc106..d67e46371 100644 --- a/tests/proc-test/package.json +++ b/tests/proc-test/package.json @@ -2,7 +2,7 @@ "name": "proc-test", "version": "0.0.1", "scripts": { - "build": "tsc -p tsconfig.json", + "build": "tsc", "test": "npx dbos rollback && npx dbos migrate && jest --detectOpenHandles", "lint": "eslint src", "lint-fix": "eslint --fix src", @@ -24,7 +24,7 @@ "typescript-eslint": "^8.0.0-alpha.30" }, "dependencies": { - "@dbos-inc/dbos-sdk": "file:../..", + "@dbos-inc/dbos-sdk": "file:../../dist", "knex": "3.1.0" } } diff --git a/tests/proc-test/tsconfig.json b/tests/proc-test/tsconfig.json index 8bd5845f0..5488d6185 100644 --- a/tests/proc-test/tsconfig.json +++ b/tests/proc-test/tsconfig.json @@ -19,6 +19,7 @@ "src" ], "exclude": [ + "**/*.test.ts", "dist" ] } From 232a68188400bd5e7dd71a1adb788b950e3e2745 Mon Sep 17 00:00:00 2001 From: Chuck B Date: Fri, 13 Dec 2024 22:33:18 -0500 Subject: [PATCH 15/39] WHAT? --- src/dbos.ts | 2 +- tests/proc-test/src/operations.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbos.ts b/src/dbos.ts index 30a32c386..86665f07f 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -815,7 +815,7 @@ export class DBOS { return decorator; } - static storedProcedure(config: StoredProcedureConfig={}) { + static storedTransaction(config: StoredProcedureConfig={}) { function decorator( target: object, propertyKey: string, diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 87618e87d..dc834bde2 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -140,7 +140,7 @@ export class StoredProcTest { return 0; } - @DBOS.storedProcedure() + @DBOS.storedTransaction() static async helloProcedure_v2(user: string): Promise { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; const { rows } = await DBOS.pgClient.query(query, [user]); From 76007007123d7999d636eaa7d820de6b7cbc8e36 Mon Sep 17 00:00:00 2001 From: Chuck B Date: Fri, 13 Dec 2024 22:39:12 -0500 Subject: [PATCH 16/39] Name back --- src/dbos.ts | 2 +- tests/proc-test/src/operations.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbos.ts b/src/dbos.ts index 86665f07f..30a32c386 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -815,7 +815,7 @@ export class DBOS { return decorator; } - static storedTransaction(config: StoredProcedureConfig={}) { + static storedProcedure(config: StoredProcedureConfig={}) { function decorator( target: object, propertyKey: string, diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index dc834bde2..87618e87d 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -140,7 +140,7 @@ export class StoredProcTest { return 0; } - @DBOS.storedTransaction() + @DBOS.storedProcedure() static async helloProcedure_v2(user: string): Promise { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; const { rows } = await DBOS.pgClient.query(query, [user]); From a4f5c64a3262bc1dc20d3b7a919a062f3724c08c Mon Sep 17 00:00:00 2001 From: Chuck B Date: Fri, 13 Dec 2024 22:49:52 -0500 Subject: [PATCH 17/39] And... back. --- src/dbos.ts | 2 +- tests/proc-test/src/operations.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbos.ts b/src/dbos.ts index 30a32c386..86665f07f 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -815,7 +815,7 @@ export class DBOS { return decorator; } - static storedProcedure(config: StoredProcedureConfig={}) { + static storedTransaction(config: StoredProcedureConfig={}) { function decorator( target: object, propertyKey: string, diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 87618e87d..dc834bde2 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -140,7 +140,7 @@ export class StoredProcTest { return 0; } - @DBOS.storedProcedure() + @DBOS.storedTransaction() static async helloProcedure_v2(user: string): Promise { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; const { rows } = await DBOS.pgClient.query(query, [user]); From a24dcc6a746b0a8de668b675708b97cbd93e05e7 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 8 Jan 2025 14:02:07 -0500 Subject: [PATCH 18/39] fix merge bug --- src/dbos.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbos.ts b/src/dbos.ts index cc04fdb8c..b1f827c1d 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -587,7 +587,7 @@ export class DBOS { ? (...args: unknown[]) => DBOSExecutor.globalInstance!.external( op.registeredFunction as StepFunction, {}, ...args) : op.procConfig - ? (...args: unknown[]) => DBOSExecutor.globalInstance!.procedure(op.registeredFunction as StoredProcedureFunc, {}, ...args) + ? (...args: unknown[]) => DBOSExecutor.globalInstance!.procedure(op.registeredFunction as StoredProcedure, {}, ...args) : undefined; } return proxy as InvokeFuncs; From c3f2c23a6095bb0023b5120a812c1d610b66ca72 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 8 Jan 2025 14:05:15 -0500 Subject: [PATCH 19/39] fix proc test dependency --- tests/proc-test/package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/proc-test/package.json b/tests/proc-test/package.json index d67e46371..f9a6f89de 100644 --- a/tests/proc-test/package.json +++ b/tests/proc-test/package.json @@ -24,7 +24,7 @@ "typescript-eslint": "^8.0.0-alpha.30" }, "dependencies": { - "@dbos-inc/dbos-sdk": "file:../../dist", + "@dbos-inc/dbos-sdk": "file:../..", "knex": "3.1.0" } } From 3f9f9586243f1dbd647086bfd1564b84fb4c6ed7 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 8 Jan 2025 16:48:00 -0500 Subject: [PATCH 20/39] v2 stored proc exec local --- src/dbos-executor.ts | 30 ++++++++++++++++------ src/dbos.ts | 2 +- tests/proc-test/src/operations.test.ts | 35 +++++++++++--------------- tests/proc-test/src/operations.ts | 8 +++--- 4 files changed, 42 insertions(+), 33 deletions(-) diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index 8ec655532..3c183af13 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -592,7 +592,7 @@ export class DBOSExecutor implements DBOSExecutorContext { throw new DBOSNotRegisteredError(`Step function name '${cfname}' is not registered.`); } - return {commInfo: stepInfo, clsInst: getConfiguredInstance(className, cfgName)}; + return { commInfo: stepInfo, clsInst: getConfiguredInstance(className, cfgName) }; } getProcedureClassName(pf: StoredProcedure) { @@ -1124,7 +1124,7 @@ export class DBOSExecutor implements DBOSExecutorContext { try { const result = executeLocally - ? await this.#callProcedureFunctionLocal(proc, args, wfCtx, span, procInfo.config, funcId) + ? await this.#callProcedureFunctionLocal(proc, args, wfCtx, span, procInfo, funcId) : await this.#callProcedureFunctionRemote(proc, args, wfCtx, span, procInfo.config, funcId); span.setStatus({ code: SpanStatusCode.OK }); return result; @@ -1137,11 +1137,18 @@ export class DBOSExecutor implements DBOSExecutorContext { } } - async #callProcedureFunctionLocal(proc: StoredProcedure, args: T, wfCtx: WorkflowContextImpl, span: Span, config: StoredProcedureConfig, funcId: number): Promise { + async #callProcedureFunctionLocal( + proc: StoredProcedure, + args: T, + wfCtx: WorkflowContextImpl, + span: Span, + procInfo: ProcedureRegInfo, + funcId: number + ): Promise { let retryWaitMillis = 1; const backoffFactor = 1.5; const maxRetryWaitMs = 2000; // Maximum wait 2 seconds. - const readOnly = config.readOnly ?? false; + const readOnly = procInfo.config.readOnly ?? false; while (true) { let txn_snapshot = "invalid"; @@ -1170,9 +1177,16 @@ export class DBOSExecutor implements DBOSExecutorContext { } let cresult: R | undefined; - await runWithStoredProcContext(ctxt, async () => { - cresult = await proc(ctxt, ...args); - }); + if (procInfo.registration.passContext) { + await runWithStoredProcContext(ctxt, async () => { + cresult = await proc(ctxt, ...args); + }); + } else { + await runWithStoredProcContext(ctxt, async () => { + const pf = proc as unknown as (...args: T)=>Promise; + cresult = await pf(...args); + }); + } const result = cresult! if (readOnly) { @@ -1197,7 +1211,7 @@ export class DBOSExecutor implements DBOSExecutorContext { }; try { - const result = await this.invokeStoredProcFunction(wrappedProcedure, { isolationLevel: config.isolationLevel }); + const result = await this.invokeStoredProcFunction(wrappedProcedure, { isolationLevel: procInfo.config.isolationLevel }); span.setStatus({ code: SpanStatusCode.OK }); return result; } catch (err) { diff --git a/src/dbos.ts b/src/dbos.ts index b1f827c1d..7e09d7edc 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -896,7 +896,7 @@ export class DBOS { return decorator; } - static storedTransaction(config: StoredProcedureConfig={}) { + static storedProcedure(config: StoredProcedureConfig={}) { function decorator( target: object, propertyKey: string, diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 84c89801f..5bf1b4844 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -175,29 +175,24 @@ describe("operations-test", () => { DBOS.setConfig(config); await DBOS.launch(); - try { - - const wfUUID = uuidv1(); - const user = `txAndProcWFv2_${Date.now()}`; - const res = await DBOS.withNextWorkflowID(wfUUID, async () => { - return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); - }) + const wfUUID = uuidv1(); + const user = `txAndProcWFv2_${Date.now()}`; + const res = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); + }) - expect(res.count).toBe(0); - expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expect(res.count).toBe(0); + expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); - expect(txRows.length).toBe(2); - expect(txRows[0].function_id).toBe(0); - expect(txRows[0].output).toBe("0"); - expectNullResult(txRows[0].error); + const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); + expect(txRows.length).toBe(2); + expect(txRows[0].function_id).toBe(0); + expect(txRows[0].output).toBe("0"); + expectNullResult(txRows[0].error); - expect(txRows[1].function_id).toBe(1); - expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - expectNullResult(txRows[1].error); - } finally { - await DBOS.shutdown(); - } + expect(txRows[1].function_id).toBe(1); + expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expectNullResult(txRows[1].error); }); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index dc834bde2..c869fe766 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -126,8 +126,8 @@ export class StoredProcTest { static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; }> { // Retrieve the number of times this user has been greeted. const count = await StoredProcTest.getGreetCountTx_v2(user); - const greeting = await DBOS.invoke(StoredProcTest).helloProcedure(user); - + const greeting = await StoredProcTest.helloProcedure_v2_local(user); + return { count, greeting }; } @@ -140,8 +140,8 @@ export class StoredProcTest { return 0; } - @DBOS.storedTransaction() - static async helloProcedure_v2(user: string): Promise { + @DBOS.storedProcedure({ executeLocally: true}) + static async helloProcedure_v2_local(user: string): Promise { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; const { rows } = await DBOS.pgClient.query(query, [user]); const greet_count = rows[0].greet_count; From c313afba5686fcdb5381dcc2424a119cc321461b Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 9 Jan 2025 13:39:38 -0500 Subject: [PATCH 21/39] more tests --- packages/dbos-compiler/tests/compiler.test.ts | 6 +- .../dbos-compiler/tests/more-compiler.test.ts | 81 ++++++++----------- packages/dbos-compiler/tests/test-code.ts | 48 +++++++++++ packages/dbos-compiler/tests/test-utility.ts | 1 + 4 files changed, 86 insertions(+), 50 deletions(-) diff --git a/packages/dbos-compiler/tests/compiler.test.ts b/packages/dbos-compiler/tests/compiler.test.ts index 939835fcf..96b0477c2 100644 --- a/packages/dbos-compiler/tests/compiler.test.ts +++ b/packages/dbos-compiler/tests/compiler.test.ts @@ -13,7 +13,7 @@ describe("compiler", () => { const testClass = file.getClassOrThrow("Test"); const methods = testClass.getStaticMethods(); - expect(methods.length).toBe(8); + expect(methods.length).toBe(16); expect(testClass.getStaticMethod("testProcedure")).toBeDefined(); }); @@ -25,7 +25,7 @@ describe("compiler", () => { const testClass = file.getClassOrThrow("Test"); const methods = testClass.getStaticMethods(); - expect(methods.length).toBe(8); + expect(methods.length).toBe(16); expect(testClass.getStaticMethod("testProcedure")).toBeDefined(); }); @@ -35,7 +35,7 @@ describe("compiler", () => { const procMethods = getProcMethods(file); - expect(procMethods.length).toBe(8); + expect(procMethods.length).toBe(16); const testClass = file.getClassOrThrow("Test"); const testProcMethod = testClass.getStaticMethodOrThrow("testProcedure"); expect(procMethods[0]).toEqual(testProcMethod); diff --git a/packages/dbos-compiler/tests/more-compiler.test.ts b/packages/dbos-compiler/tests/more-compiler.test.ts index b4c921a61..cfb9ee575 100644 --- a/packages/dbos-compiler/tests/more-compiler.test.ts +++ b/packages/dbos-compiler/tests/more-compiler.test.ts @@ -50,6 +50,14 @@ describe("more compiler", () => { testGetHandlerWorkflow_v2: "workflow", testGetHandlerTx_v2: "transaction", testGetHandlerStep_v2: "step", + testProcedure_v2: "storedProcedure", + testReadOnlyProcedure_v2: "storedProcedure", + testRepeatableReadProcedure_v2: "storedProcedure", + testConfiguredProcedure_v2: "storedProcedure", + testLocalProcedure_v2: "storedProcedure", + testLocalReadOnlyProcedure_v2: "storedProcedure", + testLocalRepeatableReadProcedure_v2: "storedProcedure", + testLocalConfiguredProcedure_v2: "storedProcedure", testStep_v2: "step", testTransaction_v2: "transaction", @@ -76,6 +84,7 @@ describe("more compiler", () => { }); describe("getDecoratorInfo", () => { + it("testGetHandler", () => { const method = cls.getStaticMethodOrThrow("testGetHandler"); @@ -122,53 +131,31 @@ describe("more compiler", () => { }) describe("getStoredProcConfig", () => { - it("testProcedure", () => { - const method = cls.getStaticMethodOrThrow("testProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({}); - }); - - it("testReadOnlyProcedure", () => { - const method = cls.getStaticMethodOrThrow("testReadOnlyProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ readOnly: true }); - }); - - it("testRepeatableReadProcedure", () => { - const method = cls.getStaticMethodOrThrow("testRepeatableReadProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ isolationLevel: "REPEATABLE READ" }); - }); - - it("testConfiguredProcedure", () => { - const method = cls.getStaticMethodOrThrow("testConfiguredProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ readOnly: true, isolationLevel: "READ COMMITTED" }); - }); - - it("testLocalProcedure", () => { - const method = cls.getStaticMethodOrThrow("testLocalProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ executeLocally: true }); - }); - - it("testLocalReadOnlyProcedure", () => { - const method = cls.getStaticMethodOrThrow("testLocalReadOnlyProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ readOnly: true, executeLocally: true }); - }); - - it("testLocalRepeatableReadProcedure", () => { - const method = cls.getStaticMethodOrThrow("testLocalRepeatableReadProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ isolationLevel: "REPEATABLE READ", executeLocally: true }); - }); - - it("testLocalConfiguredProcedure", () => { - const method = cls.getStaticMethodOrThrow("testLocalConfiguredProcedure"); - const config = getStoredProcConfig(method); - expect(config).toEqual({ readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true }); - }); + const data = { + "testProcedure": {}, + "testReadOnlyProcedure": { readOnly: true }, + "testRepeatableReadProcedure": { isolationLevel: "REPEATABLE READ" }, + "testConfiguredProcedure": { readOnly: true, isolationLevel: "READ COMMITTED" }, + "testLocalProcedure": { executeLocally: true }, + "testLocalReadOnlyProcedure": { readOnly: true, executeLocally: true }, + "testLocalRepeatableReadProcedure": { isolationLevel: "REPEATABLE READ", executeLocally: true }, + "testLocalConfiguredProcedure": { readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true } + } + + for (const [name, config] of Object.entries(data)) { + it(name, () => { + const method = cls.getStaticMethodOrThrow(name); + const actual = getStoredProcConfig(method); + expect(actual).toEqual(config); + }); + + const v2name = `${name}_v2`; + it(v2name, () => { + const method = cls.getStaticMethodOrThrow(v2name); + const actual = getStoredProcConfig(method); + expect(actual).toEqual(config); + }); + } }) }); diff --git a/packages/dbos-compiler/tests/test-code.ts b/packages/dbos-compiler/tests/test-code.ts index e2d876e5d..65b52fb7e 100644 --- a/packages/dbos-compiler/tests/test-code.ts +++ b/packages/dbos-compiler/tests/test-code.ts @@ -115,6 +115,30 @@ export class Test { @StoredProcedure({ readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true }) static async testLocalConfiguredProcedure(ctxt: StoredProcedureContext, message: string): Promise { } + @DBOS.storedProcedure() + static async testProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ readOnly: true }) + static async testReadOnlyProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ isolationLevel: "REPEATABLE READ" }) + static async testRepeatableReadProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ readOnly: true, isolationLevel: "READ COMMITTED" }) + static async testConfiguredProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ executeLocally: true}) + static async testLocalProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ readOnly: true, executeLocally: true }) + static async testLocalReadOnlyProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ isolationLevel: "REPEATABLE READ", executeLocally: true }) + static async testLocalRepeatableReadProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @DBOS.storedProcedure({ readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true }) + static async testLocalConfiguredProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + @DBOSInitializer() static async testDBOSInitializer(ctxt: InitContext): Promise { } @@ -242,6 +266,30 @@ export class Test { @TestStoredProcedure({ readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true }) static async testLocalConfiguredProcedure(ctxt: StoredProcedureContext, message: string): Promise { } + @TestDBOS.storedProcedure() + static async testProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ readOnly: true }) + static async testReadOnlyProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ isolationLevel: "REPEATABLE READ" }) + static async testRepeatableReadProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ readOnly: true, isolationLevel: "READ COMMITTED" }) + static async testConfiguredProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ executeLocally: true}) + static async testLocalProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ readOnly: true, executeLocally: true }) + static async testLocalReadOnlyProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ isolationLevel: "REPEATABLE READ", executeLocally: true }) + static async testLocalRepeatableReadProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + + @TestDBOS.storedProcedure({ readOnly: true, isolationLevel: "READ COMMITTED", executeLocally: true }) + static async testLocalConfiguredProcedure_v2(ctxt: StoredProcedureContext, message: string): Promise { } + @TestInitializer() static async testDBOSInitializer(ctxt: InitContext): Promise { } diff --git a/packages/dbos-compiler/tests/test-utility.ts b/packages/dbos-compiler/tests/test-utility.ts index a3d91d3be..2ceaab4ea 100644 --- a/packages/dbos-compiler/tests/test-utility.ts +++ b/packages/dbos-compiler/tests/test-utility.ts @@ -101,6 +101,7 @@ declare module "@dbos-inc/dbos-sdk" { static workflow(config?: WorkflowConfig); static transaction(config?: TransactionConfig); static step(config?: StepConfig); + static storedProcedure(config?: StoredProcedureConfig); static getApi(url: string); static postApi(url: string); static putApi(url: string); From 99511f380d23edd05b94d28496f583c5b8ba3f37 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 9 Jan 2025 16:05:28 -0500 Subject: [PATCH 22/39] get version info from DBOS decorators --- packages/dbos-compiler/compiler.ts | 141 +++++++++++------- .../dbos-compiler/tests/more-compiler.test.ts | 107 ++++++------- packages/dbos-compiler/tests/test-code.ts | 14 ++ 3 files changed, 149 insertions(+), 113 deletions(-) diff --git a/packages/dbos-compiler/compiler.ts b/packages/dbos-compiler/compiler.ts index 195d16c83..d727ad61c 100644 --- a/packages/dbos-compiler/compiler.ts +++ b/packages/dbos-compiler/compiler.ts @@ -136,8 +136,10 @@ export function checkStoredProcConfig(methods: readonly (readonly [tsm.MethodDec function getStoredProcDecorator(method: tsm.MethodDeclaration) { for (const decorator of method.getDecorators()) { - const kind = getDbosDecoratorKind(decorator); - if (kind === "storedProcedure") { return decorator; } + const info = getDbosDecoratorInfo(decorator); + if (info?.kind === "storedProcedure") { + return decorator; + } } } } @@ -147,8 +149,9 @@ export function removeDbosMethods(file: tsm.SourceFile) { if (tsm.Node.isClassDeclaration(node)) { traversal.skip(); for (const method of node.getStaticMethods()) { - const kind = getDbosMethodKind(method); - switch (kind) { + const info = getDbosMethodInfo(method); + if (!info) { continue; } + switch (info.kind) { case 'workflow': case 'step': case 'initializer': @@ -158,12 +161,11 @@ export function removeDbosMethods(file: tsm.SourceFile) { break; } case 'storedProcedure': - case undefined: break; default: { - const _: never = kind; + const _never: never = info.kind; // eslint-disable-next-line @typescript-eslint/restrict-template-expressions - throw new Error(`Unexpected DBOS method kind: ${kind}`); + throw new Error(`Unexpected DBOS method kind: ${info}`); } } } @@ -177,8 +179,8 @@ export function getProcMethods(file: tsm.SourceFile) { if (tsm.Node.isClassDeclaration(node)) { traversal.skip(); for (const method of node.getStaticMethods()) { - const kind = getDbosMethodKind(method); - if (kind === 'storedProcedure') { + const info = getDbosMethodInfo(method); + if (info?.kind === 'storedProcedure') { methods.push(method); } } @@ -310,7 +312,8 @@ function deAsync(project: tsm.Project) { sourceFile.forEachChild(node => { if (tsm.Node.isClassDeclaration(node)) { for (const method of node.getStaticMethods()) { - if (getDbosMethodKind(method) === 'storedProcedure') { + const info = getDbosMethodInfo(method) + if (info?.kind === 'storedProcedure') { method.setIsAsync(false); method.getBody()?.transform(traversal => { const node = traversal.visitChildren(); @@ -328,6 +331,12 @@ function deAsync(project: tsm.Project) { function isValid(value: T | null | undefined): value is T { return !!value; } type DbosDecoratorKind = "handler" | "storedProcedure" | "transaction" | "workflow" | "step" | "initializer"; +type DbosDecoratorVersion = 1 | 2; + +interface DbosDecoratorInfo { + kind: DbosDecoratorKind; + version: DbosDecoratorVersion; +} export function getImportSpecifier(node: tsm.Identifier | undefined): tsm.ImportSpecifier | undefined { const symbol = node?.getSymbol(); @@ -343,12 +352,12 @@ export function getImportSpecifier(node: tsm.Identifier | undefined): tsm.Import return undefined; } -function isDbosImport(node: tsm.ImportSpecifier ): boolean { +function isDbosImport(node: tsm.ImportSpecifier): boolean { const modSpec = node.getImportDeclaration().getModuleSpecifier(); return modSpec.getLiteralText() === "@dbos-inc/dbos-sdk"; } -function getDbosDecoratorKind(node: tsm.Decorator): DbosDecoratorKind | undefined { +function getDbosDecoratorInfo(node: tsm.Decorator): DbosDecoratorInfo | undefined { if (!node.isDecoratorFactory()) { return undefined; } const expr = node.getCallExpressionOrThrow().getExpression(); @@ -357,24 +366,8 @@ function getDbosDecoratorKind(node: tsm.Decorator): DbosDecoratorKind | undefine if (tsm.Node.isIdentifier(expr)) { const impSpec = getImportSpecifier(expr); if (impSpec && isDbosImport(impSpec)) { - const { name } = impSpec.getStructure(); - switch (name) { - case "GetApi": - case "PostApi": - case "PutApi": - case "PatchApi": - case "DeleteApi": - return "handler"; - case "StoredProcedure": return "storedProcedure"; - case "Transaction": return "transaction"; - case "Workflow": return "workflow"; - case "Communicator": - case "Step": - return "step"; - case "DBOSInitializer": - case "DBOSDeploy": - return "initializer"; - } + const kind = getImportSpecifierStructureKind(impSpec.getStructure()); + if (kind) { return { kind, version: 1 }; } } } @@ -383,55 +376,90 @@ function getDbosDecoratorKind(node: tsm.Decorator): DbosDecoratorKind | undefine const impSpec = getImportSpecifier(expr.getExpressionIfKind(tsm.SyntaxKind.Identifier)); if (impSpec && isDbosImport(impSpec)) { const { name } = impSpec.getStructure(); - if (name === "DBOS") { - switch (expr.getName()) { - case "getApi": - case "postApi": - case "putApi": - case "patchApi": - case "deleteApi": - return "handler"; - case "workflow": return "workflow"; - case "transaction": return "transaction"; - case "step": return "step"; - case "storedProcedure": return "storedProcedure"; - } + if (name === "DBOS") { + const kind = getPropertyAccessExpressionKind(expr); + if (kind) { return { kind, version: 2 }; } } } } return undefined; + + function getImportSpecifierStructureKind( + { name }: tsm.ImportSpecifierStructure + ): DbosDecoratorKind | undefined { + switch (name) { + case "GetApi": + case "PostApi": + case "PutApi": + case "PatchApi": + case "DeleteApi": + return "handler"; + case "StoredProcedure": return "storedProcedure"; + case "Transaction": return "transaction"; + case "Workflow": return "workflow"; + case "Communicator": + case "Step": + return "step"; + case "DBOSInitializer": + case "DBOSDeploy": + return "initializer"; + default: return undefined; + } + } + + function getPropertyAccessExpressionKind( + node: tsm.PropertyAccessExpression + ): DbosDecoratorKind | undefined { + switch (node.getName()) { + case "getApi": + case "postApi": + case "putApi": + case "patchApi": + case "deleteApi": + return "handler"; + case "workflow": return "workflow"; + case "transaction": return "transaction"; + case "step": return "step"; + case "storedProcedure": return "storedProcedure"; + default: return undefined; + } + } } // helper function to determine the kind of DBOS method -export function getDbosMethodKind(node: tsm.MethodDeclaration): DbosDecoratorKind | undefined { +export function getDbosMethodInfo(node: tsm.MethodDeclaration): DbosDecoratorInfo | undefined { // Note, other DBOS method decorators (Scheduled, KafkaConsume, RequiredRole) modify runtime behavior // of DBOS methods, but are not their own unique kind. // Get/PostApi decorators are atypical in that they can be used on @Step/@Transaction/@Workflow // methods as well as on their own. - let isHandler = false; + let handlerVersion: DbosDecoratorVersion | undefined = undefined; for (const decorator of node.getDecorators()) { - const kind = getDbosDecoratorKind(decorator); - switch (kind) { + const info = getDbosDecoratorInfo(decorator); + if (!info) { continue; } + switch (info.kind) { case "storedProcedure": case "transaction": case "workflow": case "step": case "initializer": - return kind; + return info; case "handler": - isHandler = true; - break; - case undefined: + if (handlerVersion !== undefined) { + throw new Error("Multiple handler decorators"); + } + handlerVersion = info.version; break; default: { - const _never: never = kind; + const _never: never = info.kind; // eslint-disable-next-line @typescript-eslint/restrict-template-expressions - throw new Error(`Unexpected DBOS method kind: ${kind}`); + throw new Error(`Unexpected DBOS method kind: ${info.kind}`); } } } - return isHandler ? "handler" : undefined; + return handlerVersion + ? { kind: "handler", version: handlerVersion } + : undefined; } export type DecoratorArgument = boolean | string | number | DecoratorArgument[] | Record; @@ -471,7 +499,10 @@ export function parseDecoratorArgument(node: tsm.Node): DecoratorArgument { export function getStoredProcConfig(node: tsm.MethodDeclaration): StoredProcedureConfig { const decorators = node.getDecorators(); - const procDecorator = decorators.find(d => getDbosDecoratorKind(d) === "storedProcedure"); + const procDecorator = decorators.find(d => { + const info = getDbosDecoratorInfo(d); + return info?.kind === "storedProcedure"; + }); if (!procDecorator) { throw new Error("Missing StoredProcedure decorator"); } const arg0 = procDecorator.getCallExpression()?.getArguments()[0] ?? undefined; diff --git a/packages/dbos-compiler/tests/more-compiler.test.ts b/packages/dbos-compiler/tests/more-compiler.test.ts index cfb9ee575..b54ca6e5a 100644 --- a/packages/dbos-compiler/tests/more-compiler.test.ts +++ b/packages/dbos-compiler/tests/more-compiler.test.ts @@ -1,70 +1,61 @@ import * as tsm from "ts-morph"; -import { DecoratorArgument, getDbosMethodKind, getImportSpecifier, getStoredProcConfig, parseDecoratorArgument } from "../compiler.js"; +import { DecoratorArgument, getDbosMethodInfo, getImportSpecifier, getStoredProcConfig, parseDecoratorArgument } from "../compiler.js"; import { sampleDbosClass, sampleDbosClassAliased } from "./test-code.js"; import { makeTestProject } from "./test-utility.js"; import { describe, it, expect } from 'vitest'; describe("more compiler", () => { const { project } = makeTestProject(sampleDbosClass); + const { project: aliasProject } = makeTestProject(sampleDbosClassAliased); const file = project.getSourceFileOrThrow("operations.ts"); const cls = file.getClassOrThrow("Test"); - const { project: aliasProject } = makeTestProject(sampleDbosClassAliased); + function testProject(name: string, project: tsm.Project) { + it(`getDbosMethodKind ${name}`, () => { + const file = project.getSourceFileOrThrow("operations.ts"); + const cls = file.getClassOrThrow("Test"); + const entries = cls.getStaticMethods().map(m => [m.getName(), getDbosMethodInfo(m)]); + // eslint-disable-next-line @typescript-eslint/no-unsafe-assignment + const actual = Object.fromEntries(entries); + + const raw_expected = { + testGetHandler: "handler", + testPostHandler: "handler", + testDeleteHandler: "handler", + testPutHandler: "handler", + testPatchHandler: "handler", + testGetHandlerWorkflow: "workflow", + testGetHandlerTx: "transaction", + testGetHandlerComm: "step", + testGetHandlerStep: "step", + testWorkflow: "workflow", + testCommunicator: "step", + testStep: "step", + testTransaction: "transaction", + testProcedure: "storedProcedure", + testReadOnlyProcedure: "storedProcedure", + testRepeatableReadProcedure: "storedProcedure", + testConfiguredProcedure: "storedProcedure", + testLocalProcedure: "storedProcedure", + testLocalReadOnlyProcedure: "storedProcedure", + testLocalRepeatableReadProcedure: "storedProcedure", + testLocalConfiguredProcedure: "storedProcedure", + testDBOSInitializer: "initializer", + testDBOSDeploy: "initializer", + }; + + const expected = Object.fromEntries(Object.entries(raw_expected).flatMap(([key, value]) => { + const v1 = [key, { kind: value, version: 1 }]; + const v2 = [`${key}_v2`, { kind: value, version: 2 }]; + return value === "initializer" ? [v1] : [v1, v2]; + })); + + expect(actual).toEqual(expected); + }) + } - it.each([project, aliasProject])("getDbosMethodKind", (project: tsm.Project) => { - const file = project.getSourceFileOrThrow("operations.ts"); - const cls = file.getClassOrThrow("Test"); - const entries = cls.getStaticMethods().map(m => [m.getName(), getDbosMethodKind(m)]); - // eslint-disable-next-line @typescript-eslint/no-unsafe-assignment - const actual = Object.fromEntries(entries); - const expected = { - testGetHandler: "handler", - testPostHandler: "handler", - testDeleteHandler: "handler", - testPutHandler: "handler", - testPatchHandler: "handler", - testGetHandlerWorkflow: "workflow", - testGetHandlerTx: "transaction", - testGetHandlerComm: "step", - testGetHandlerStep: "step", - testWorkflow: "workflow", - testCommunicator: "step", - testStep: "step", - testTransaction: "transaction", - testProcedure: "storedProcedure", - testReadOnlyProcedure: "storedProcedure", - testRepeatableReadProcedure: "storedProcedure", - testConfiguredProcedure: "storedProcedure", - testLocalProcedure: "storedProcedure", - testLocalReadOnlyProcedure: "storedProcedure", - testLocalRepeatableReadProcedure: "storedProcedure", - testLocalConfiguredProcedure: "storedProcedure", - testDBOSInitializer: "initializer", - testDBOSDeploy: "initializer", - - testGetHandler_v2: "handler", - testPostHandler_v2: "handler", - testDeleteHandler_v2: "handler", - testPutHandler_v2: "handler", - testPatchHandler_v2: "handler", - testGetHandlerWorkflow_v2: "workflow", - testGetHandlerTx_v2: "transaction", - testGetHandlerStep_v2: "step", - testProcedure_v2: "storedProcedure", - testReadOnlyProcedure_v2: "storedProcedure", - testRepeatableReadProcedure_v2: "storedProcedure", - testConfiguredProcedure_v2: "storedProcedure", - testLocalProcedure_v2: "storedProcedure", - testLocalReadOnlyProcedure_v2: "storedProcedure", - testLocalRepeatableReadProcedure_v2: "storedProcedure", - testLocalConfiguredProcedure_v2: "storedProcedure", - - testStep_v2: "step", - testTransaction_v2: "transaction", - testWorkflow_v2: "workflow", - }; - expect(actual).toEqual(expected); - }); + testProject("project", project); + testProject("aliasProject", aliasProject); describe("aliased getDecoratorInfo", () => { it("testGetHandler", () => { @@ -164,7 +155,7 @@ interface DecoratorInfo { alias?: string; module?: string; args: DecoratorArgument[]; - } +} function testDecorators(expected: DecoratorInfo[], actual: tsm.Decorator[]) { @@ -178,7 +169,7 @@ function testDecorator(expected: DecoratorInfo, actual: tsm.Decorator) { const callExpr = actual.getCallExpressionOrThrow(); const expr = callExpr.getExpression(); - const idExpr = tsm.Node.isIdentifier(expr) + const idExpr = tsm.Node.isIdentifier(expr) ? expr : expr.asKindOrThrow(tsm.SyntaxKind.PropertyAccessExpression).getExpressionIfKindOrThrow(tsm.SyntaxKind.Identifier); diff --git a/packages/dbos-compiler/tests/test-code.ts b/packages/dbos-compiler/tests/test-code.ts index 65b52fb7e..4a5f5d4c2 100644 --- a/packages/dbos-compiler/tests/test-code.ts +++ b/packages/dbos-compiler/tests/test-code.ts @@ -66,6 +66,10 @@ export class Test { @DBOS.transaction() static async testGetHandlerTx_v2(): Promise { } + @GetApi('/test') + @DBOS.step() + static async testGetHandlerComm_v2(ctxt: CommunicatorContext): Promise { } + @DBOS.getApi('/test') @DBOS.step() static async testGetHandlerStep_v2(): Promise { } @@ -88,6 +92,9 @@ export class Test { @DBOS.step() static async testStep_v2(message: string): Promise { } + @DBOS.step() + static async testCommunicator_v2(message: string): Promise { } + @DBOS.transaction() static async testTransaction_v2(message: string): Promise { } @@ -205,6 +212,10 @@ export class Test { @TestCommunicator() static async testGetHandlerComm(ctxt: HandlerContext): Promise { } + @TestGetApi('/test') + @TestDBOS.step() + static async testGetHandlerComm_v2(ctxt: CommunicatorContext): Promise { } + @TestGetApi('/test') @TestStep() static async testGetHandlerStep(ctxt: StepContext): Promise { } @@ -239,6 +250,9 @@ export class Test { @TestDBOS.step() static async testStep_v2(message: string): Promise { } + @TestDBOS.step() + static async testCommunicator_v2(message: string): Promise { } + @TestDBOS.transaction() static async testTransaction_v2(message: string): Promise { } From f7414bd72b9093031e10f4b7ddc1ec25f689a138 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Thu, 9 Jan 2025 16:18:23 -0500 Subject: [PATCH 23/39] include version info in method info returned from compile --- packages/dbos-compiler/compiler.ts | 13 +++++++------ packages/dbos-compiler/tests/compiler.test.ts | 6 +++--- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/packages/dbos-compiler/compiler.ts b/packages/dbos-compiler/compiler.ts index d727ad61c..70175b65b 100644 --- a/packages/dbos-compiler/compiler.ts +++ b/packages/dbos-compiler/compiler.ts @@ -1,8 +1,9 @@ import tsm from 'ts-morph'; +type CompileMethodInfo = readonly [tsm.MethodDeclaration, StoredProcedureConfig, DbosDecoratorVersion]; export type CompileResult = { project: tsm.Project; - methods: (readonly [tsm.MethodDeclaration, StoredProcedureConfig])[]; + methods: CompileMethodInfo[]; }; export type IsolationLevel = "READ UNCOMMITTED" | "READ COMMITTED" | "REPEATABLE READ" | "SERIALIZABLE"; @@ -44,7 +45,7 @@ export function compile(configFileOrProject: string | tsm.Project, suppressWarni const methods = project.getSourceFiles() .flatMap(getProcMethods) - .map(m => [m, getStoredProcConfig(m)] as const); + .map(([m, v]) => [m, getStoredProcConfig(m), v] as const); diags.push(...checkStoredProcNames(methods.map(([m]) => m))); diags.push(...checkStoredProcConfig(methods, false)); @@ -121,7 +122,7 @@ export function checkStoredProcNames(methods: readonly tsm.MethodDeclaration[]): return diags; } -export function checkStoredProcConfig(methods: readonly (readonly [tsm.MethodDeclaration, StoredProcedureConfig])[], error: boolean = false): readonly tsm.ts.Diagnostic[] { +export function checkStoredProcConfig(methods: readonly CompileMethodInfo[], error: boolean = false): readonly tsm.ts.Diagnostic[] { const category = error ? tsm.ts.DiagnosticCategory.Error : tsm.ts.DiagnosticCategory.Warning; const diags = new Array(); for (const [method, config] of methods) { @@ -174,14 +175,14 @@ export function removeDbosMethods(file: tsm.SourceFile) { } export function getProcMethods(file: tsm.SourceFile) { - const methods = new Array(); + const methods = new Array<[tsm.MethodDeclaration, DbosDecoratorVersion]>(); file.forEachDescendant((node, traversal) => { if (tsm.Node.isClassDeclaration(node)) { traversal.skip(); for (const method of node.getStaticMethods()) { const info = getDbosMethodInfo(method); if (info?.kind === 'storedProcedure') { - methods.push(method); + methods.push([method, info.version]); } } } @@ -192,7 +193,7 @@ export function getProcMethods(file: tsm.SourceFile) { function getProcMethodDeclarations(file: tsm.SourceFile) { // initialize set of declarations with all tx methods and their class declaration parents const declSet = new Set(); - for (const method of getProcMethods(file)) { + for (const [method, _version] of getProcMethods(file)) { declSet.add(method); const parent = method.getParentIfKind(tsm.SyntaxKind.ClassDeclaration); if (parent) { declSet.add(parent); } diff --git a/packages/dbos-compiler/tests/compiler.test.ts b/packages/dbos-compiler/tests/compiler.test.ts index 96b0477c2..9d2990747 100644 --- a/packages/dbos-compiler/tests/compiler.test.ts +++ b/packages/dbos-compiler/tests/compiler.test.ts @@ -38,7 +38,7 @@ describe("compiler", () => { expect(procMethods.length).toBe(16); const testClass = file.getClassOrThrow("Test"); const testProcMethod = testClass.getStaticMethodOrThrow("testProcedure"); - expect(procMethods[0]).toEqual(testProcMethod); + expect(procMethods[0][0]).toEqual(testProcMethod); }); it("removeDecorators", () => { @@ -118,7 +118,7 @@ describe("compiler", () => { const procMethods = getProcMethods(file); expect(procMethods.length).toBe(1); - const diags = checkStoredProcNames(procMethods); + const diags = checkStoredProcNames(procMethods.map(([m, v]) => m)); expect(diags.length).toBe(1); expect(diags[0].category === tsm.DiagnosticCategory.Error); }); @@ -133,7 +133,7 @@ describe("compiler", () => { }`; const { project } = makeTestProject(executeLocallyFile); const file = project.getSourceFileOrThrow("operations.ts"); - const procMethods = getProcMethods(file).map(m => [m, getStoredProcConfig(m)] as const); + const procMethods = getProcMethods(file).map(([m, v]) => [m, getStoredProcConfig(m), v] as const); expect(procMethods.length).toBe(1); expect(procMethods[0][1].executeLocally).toBe(true); From a4c1f91322ec807fd1073707bde6b45da62b1d3e Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Fri, 10 Jan 2025 14:44:31 -0500 Subject: [PATCH 24/39] move api version to StoredProcConfig interface --- packages/dbos-compiler/compiler.ts | 11 ++++++----- packages/dbos-compiler/tests/compiler.test.ts | 2 +- packages/dbos-compiler/tests/more-compiler.test.ts | 10 ++++++---- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/packages/dbos-compiler/compiler.ts b/packages/dbos-compiler/compiler.ts index 70175b65b..a18fd61b4 100644 --- a/packages/dbos-compiler/compiler.ts +++ b/packages/dbos-compiler/compiler.ts @@ -1,6 +1,6 @@ import tsm from 'ts-morph'; -type CompileMethodInfo = readonly [tsm.MethodDeclaration, StoredProcedureConfig, DbosDecoratorVersion]; +type CompileMethodInfo = readonly [tsm.MethodDeclaration, StoredProcedureConfig]; export type CompileResult = { project: tsm.Project; methods: CompileMethodInfo[]; @@ -10,7 +10,8 @@ export type IsolationLevel = "READ UNCOMMITTED" | "READ COMMITTED" | "REPEATABLE export interface StoredProcedureConfig { isolationLevel?: IsolationLevel; readOnly?: boolean; - executeLocally?: boolean + executeLocally?: boolean; + version: DbosDecoratorVersion; } function hasError(diags: readonly tsm.ts.Diagnostic[]) { @@ -45,7 +46,7 @@ export function compile(configFileOrProject: string | tsm.Project, suppressWarni const methods = project.getSourceFiles() .flatMap(getProcMethods) - .map(([m, v]) => [m, getStoredProcConfig(m), v] as const); + .map(([m, v]) => [m, getStoredProcConfig(m, v)] as const); diags.push(...checkStoredProcNames(methods.map(([m]) => m))); diags.push(...checkStoredProcConfig(methods, false)); @@ -498,7 +499,7 @@ export function parseDecoratorArgument(node: tsm.Node): DecoratorArgument { } } -export function getStoredProcConfig(node: tsm.MethodDeclaration): StoredProcedureConfig { +export function getStoredProcConfig(node: tsm.MethodDeclaration, version: DbosDecoratorVersion): StoredProcedureConfig { const decorators = node.getDecorators(); const procDecorator = decorators.find(d => { const info = getDbosDecoratorInfo(d); @@ -511,5 +512,5 @@ export function getStoredProcConfig(node: tsm.MethodDeclaration): StoredProcedur const readOnly = configArg?.readOnly; const executeLocally = configArg?.executeLocally; const isolationLevel = configArg?.isolationLevel; - return { isolationLevel, readOnly, executeLocally }; + return { isolationLevel, readOnly, executeLocally, version }; } diff --git a/packages/dbos-compiler/tests/compiler.test.ts b/packages/dbos-compiler/tests/compiler.test.ts index 9d2990747..809fa877e 100644 --- a/packages/dbos-compiler/tests/compiler.test.ts +++ b/packages/dbos-compiler/tests/compiler.test.ts @@ -133,7 +133,7 @@ describe("compiler", () => { }`; const { project } = makeTestProject(executeLocallyFile); const file = project.getSourceFileOrThrow("operations.ts"); - const procMethods = getProcMethods(file).map(([m, v]) => [m, getStoredProcConfig(m), v] as const); + const procMethods = getProcMethods(file).map(([m, v]) => [m, getStoredProcConfig(m, v)] as const); expect(procMethods.length).toBe(1); expect(procMethods[0][1].executeLocally).toBe(true); diff --git a/packages/dbos-compiler/tests/more-compiler.test.ts b/packages/dbos-compiler/tests/more-compiler.test.ts index b54ca6e5a..3abf0cdfe 100644 --- a/packages/dbos-compiler/tests/more-compiler.test.ts +++ b/packages/dbos-compiler/tests/more-compiler.test.ts @@ -135,16 +135,18 @@ describe("more compiler", () => { for (const [name, config] of Object.entries(data)) { it(name, () => { + const expected = { ...config, version: 1 }; const method = cls.getStaticMethodOrThrow(name); - const actual = getStoredProcConfig(method); - expect(actual).toEqual(config); + const actual = getStoredProcConfig(method, 1); + expect(actual).toEqual(expected); }); const v2name = `${name}_v2`; it(v2name, () => { + const expected = { ...config, version: 2 }; const method = cls.getStaticMethodOrThrow(v2name); - const actual = getStoredProcConfig(method); - expect(actual).toEqual(config); + const actual = getStoredProcConfig(method, 2); + expect(actual).toEqual(expected); }); } }) From b24a887eca580465840db9b7fcff3b038c055bfd Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Fri, 10 Jan 2025 15:15:00 -0500 Subject: [PATCH 25/39] update proc test --- tests/proc-test/src/operations.ts | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index c869fe766..8914e61aa 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -9,11 +9,6 @@ export interface dbos_hello { export class StoredProcTest { - // @GetApi('/greeting/:user') // Serve this function from HTTP GET requests to the /greeting endpoint with 'user' as a path parameter - // static async helloHandler(context: HandlerContext, @ArgSource(ArgSources.URL) user: string) { - // return await context.invoke(Hello).helloProcedure(user); - // } - @StoredProcedure({ readOnly: true }) static async getGreetCount(ctxt: StoredProcedureContext, user: string): Promise { const query = "SELECT greet_count FROM dbos_hello WHERE name = $1;"; From e1b65d7dff2fcbd813865fd1c6d52382ce8664e7 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Fri, 10 Jan 2025 15:30:16 -0500 Subject: [PATCH 26/39] fix compile bug --- src/dbos.ts | 111 +++++++++++++++++++++++----------------------------- 1 file changed, 50 insertions(+), 61 deletions(-) diff --git a/src/dbos.ts b/src/dbos.ts index 781a63b9a..94d2c9eaa 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -415,7 +415,7 @@ export class DBOS { return this.sleepms(durationMS); } - static async withNextWorkflowID(wfid: string, callback: ()=>Promise) : Promise { + static async withNextWorkflowID(wfid: string, callback: () => Promise): Promise { const pctx = getCurrentContextStore(); if (pctx) { const pcwfid = pctx.idAssignedForNextWorkflow; @@ -465,7 +465,7 @@ export class DBOS { } } - static async withWorkflowQueue(wfq: string, callback: ()=>Promise) : Promise { + static async withWorkflowQueue(wfq: string, callback: () => Promise): Promise { const pctx = getCurrentContextStore(); if (pctx) { const pcwfq = pctx.queueAssignedForWorkflows; @@ -478,7 +478,7 @@ export class DBOS { } } else { - return runWithTopContext({queueAssignedForWorkflows: wfq}, callback); + return runWithTopContext({ queueAssignedForWorkflows: wfq }, callback); } } @@ -494,8 +494,7 @@ export class DBOS { } static proxyInvokeWF(object: T, configuredInstance: ConfiguredInstance | null, inParams?: StartWorkflowParams): - InvokeFunctionsAsync - { + InvokeFunctionsAsync { const ops = getRegisteredOperations(object); const proxy: Record = {}; @@ -509,7 +508,7 @@ export class DBOS { const funcId = wfctx.functionIDGetIncrement(); wfId = wfId || (wfctx.workflowUUID + "-" + funcId); - const wfParams : WorkflowParams = { + const wfParams: WorkflowParams = { workflowUUID: wfId, parentCtx: wfctx, configuredInstance, @@ -519,8 +518,8 @@ export class DBOS { for (const op of ops) { proxy[op.name] = op.workflowConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.internalWorkflow( - (op.registeredFunction as WorkflowFunction), - wfParams, wfctx.workflowUUID, funcId, ...args) + (op.registeredFunction as WorkflowFunction), + wfParams, wfctx.workflowUUID, funcId, ...args) : undefined; } @@ -534,7 +533,7 @@ export class DBOS { let span = pctx.span; if (!span) { span = DBOS.executor.tracer.startSpan( - pctx.operationCaller || "startWorkflow", + pctx.operationCaller || "startWorkflow", { operationUUID: wfId, operationType: pctx.operationType, @@ -544,7 +543,7 @@ export class DBOS { }, ); } - parentCtx = new DBOSContextImpl(pctx.operationCaller || "startWorkflow", span, DBOS.logger as GlobalLogger); + parentCtx = new DBOSContextImpl(pctx.operationCaller || "startWorkflow", span, DBOS.logger as GlobalLogger); parentCtx.request = pctx.request || {}; parentCtx.authenticatedUser = pctx.authenticatedUser || ""; parentCtx.assumedRole = pctx.assumedRole || ""; @@ -577,7 +576,7 @@ export class DBOS { // Run the temp workflow way... if (typeof object === 'function') { const ops = getRegisteredOperations(object); - + const proxy: Record = {}; for (const op of ops) { proxy[op.name] = op.txnConfig @@ -595,19 +594,19 @@ export class DBOS { else { const targetInst = object as ConfiguredInstance; const ops = getRegisteredOperations(targetInst); - + const proxy: Record = {}; for (const op of ops) { proxy[op.name] = op.txnConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.transaction( - op.registeredFunction as TransactionFunction, {configuredInstance: targetInst}, ...args) + op.registeredFunction as TransactionFunction, { configuredInstance: targetInst }, ...args) : op.commConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.external( - op.registeredFunction as StepFunction, {configuredInstance: targetInst}, ...args) + op.registeredFunction as StepFunction, { configuredInstance: targetInst }, ...args) : undefined; } return proxy as InvokeFuncsInst; - } + } } const wfctx = assertCurrentWorkflowContext(); if (typeof object === 'function') { @@ -617,13 +616,13 @@ export class DBOS { for (const op of ops) { proxy[op.name] = op.txnConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callTransactionFunction( - op.registeredFunction as TransactionFunction, null, wfctx, ...args) + op.registeredFunction as TransactionFunction, null, wfctx, ...args) : op.commConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callStepFunction( - op.registeredFunction as StepFunction, null, wfctx, ...args) + op.registeredFunction as StepFunction, null, wfctx, ...args) : op.procConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callProcedureFunction( - op.registeredFunction as StoredProcedure, wfctx, ...args) + op.registeredFunction as StoredProcedure, wfctx, ...args) : undefined; } return proxy as InvokeFuncs; @@ -636,7 +635,7 @@ export class DBOS { for (const op of ops) { proxy[op.name] = op.txnConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callTransactionFunction( - op.registeredFunction as TransactionFunction, targetInst, wfctx, ...args) + op.registeredFunction as TransactionFunction, targetInst, wfctx, ...args) : op.commConfig ? (...args: unknown[]) => DBOSExecutor.globalInstance!.callStepFunction( op.registeredFunction as StepFunction, targetInst, wfctx, ...args) @@ -707,19 +706,17 @@ export class DBOS { return scheddec; } - static workflow(config: WorkflowConfig={}) - { - function decorator < + static workflow(config: WorkflowConfig = {}) { + function decorator< This, Args extends unknown[], Return > - ( - target: object, - propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise> - ) - { + ( + target: object, + propertyKey: string, + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise> + ) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.workflowConfig = config; @@ -765,16 +762,16 @@ export class DBOS { // If pctx has no span, e.g., has not been setup through `withTracedContext`, set up a parent span for the workflow here. let span = pctx.span; if (!span) { - span = DBOS.executor.tracer.startSpan( - pctx.operationCaller || "workflowCaller", - { - operationUUID: wfId, - operationType: pctx.operationType, - authenticatedUser: pctx.authenticatedUser, - assumedRole: pctx.assumedRole, - authenticatedRoles: pctx.authenticatedRoles, - }, - ); + span = DBOS.executor.tracer.startSpan( + pctx.operationCaller || "workflowCaller", + { + operationUUID: wfId, + operationType: pctx.operationType, + authenticatedUser: pctx.authenticatedUser, + assumedRole: pctx.assumedRole, + authenticatedRoles: pctx.authenticatedRoles, + }, + ); } parentCtx = new DBOSContextImpl(pctx.operationCaller || "workflowCaller", span, DBOS.logger as GlobalLogger); parentCtx.request = pctx.request || {}; @@ -787,7 +784,7 @@ export class DBOS { const wfParams: InternalWorkflowParams = { workflowUUID: wfId, queueName: pctx?.queueAssignedForWorkflows, - configuredInstance : inst, + configuredInstance: inst, parentCtx, }; @@ -813,12 +810,11 @@ export class DBOS { return decorator; } - static transaction(config: TransactionConfig={}) { + static transaction(config: TransactionConfig = {}) { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) - { + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.txnConfig = config; @@ -896,13 +892,12 @@ export class DBOS { return decorator; } - static storedProcedure(config: StoredProcedureConfig={}) { + static storedProcedure(config: StoredProcedureConfig = {}) { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) - { - const { descriptor, registration } = registerAndWrapContextFreeFunction(target, propertyKey, inDescriptor); + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { + const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.procConfig = config; const invokeWrapper = async function (this: This, ...rawArgs: Args): Promise { @@ -947,12 +942,11 @@ export class DBOS { return decorator; } - static step(config: StepConfig={}) { + static step(config: StepConfig = {}) { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) - { + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.commConfig = config; @@ -1011,7 +1005,7 @@ export class DBOS { parentCtx, }; - return await DBOS.executor.external( + return await DBOS.executor.external( registration.registeredFunction as unknown as StepFunction, wfParams, ...rawArgs ); @@ -1051,8 +1045,7 @@ export class DBOS { static defaultRequiredRole(anyOf: string[]) { // eslint-disable-next-line @typescript-eslint/no-explicit-any - function clsdec(ctor: T) - { + function clsdec(ctor: T) { const clsreg = getOrCreateClassRegistration(ctor); clsreg.requiredRole = anyOf; } @@ -1063,9 +1056,8 @@ export class DBOS { function apidec( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) - { - const {descriptor, registration} = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { + const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.requiredRole = anyOf; return descriptor; @@ -1076,8 +1068,7 @@ export class DBOS { ///// // Registration, etc ///// - static configureInstance(cls: new (name:string, ...args: T) => R, name: string, ...args: T) : R - { + static configureInstance(cls: new (name: string, ...args: T) => R, name: string, ...args: T): R { return configureInstance(cls, name, ...args); } @@ -1086,13 +1077,11 @@ export class DBOS { target: object, propertyKey: string, descriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>, - ) - { + ) { return registerAndWrapDBOSFunction(target, propertyKey, descriptor); } - static async executeWorkflowById(workflowId: string, startNewWorkflow: boolean = false): Promise> - { + static async executeWorkflowById(workflowId: string, startNewWorkflow: boolean = false): Promise> { if (!DBOSExecutor.globalInstance) { throw new DBOSExecutorNotInitializedError(); } From 455daefdb4e0a649b84246572c79f0641e10449b Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Fri, 10 Jan 2025 16:44:12 -0500 Subject: [PATCH 27/39] initial v2 proc generation --- packages/dbos-compiler/generator.ts | 10 +++-- .../templates/dbos.create.liquid | 28 ++++++++++++ .../templates/method.create.liquid | 9 ++-- tests/proc-test/src/operations.test.ts | 45 ++++++++++--------- tests/proc-test/src/operations.ts | 13 +++++- 5 files changed, 76 insertions(+), 29 deletions(-) diff --git a/packages/dbos-compiler/generator.ts b/packages/dbos-compiler/generator.ts index 92f46c085..f4b18cb23 100644 --- a/packages/dbos-compiler/generator.ts +++ b/packages/dbos-compiler/generator.ts @@ -66,9 +66,10 @@ async function generateDbosDrop(appVersion: string | undefined) { } function getMethodContext(method: tsm.MethodDeclaration, config: StoredProcedureConfig, appVersion: string | undefined) { - const readOnly = config?.readOnly ?? false; - const executeLocally = config?.executeLocally ?? false; - const isolationLevel = config?.isolationLevel ?? "SERIALIZABLE"; + const readOnly = config.readOnly ?? false; + const executeLocally = config.executeLocally ?? false; + const isolationLevel = config.isolationLevel ?? "SERIALIZABLE"; + const apiVersion = config.version; const methodName = method.getName(); const className = method.getParentIfKindOrThrow(tsm.SyntaxKind.ClassDeclaration).getName(); @@ -82,7 +83,8 @@ function getMethodContext(method: tsm.MethodDeclaration, config: StoredProcedure methodName, className, moduleName, - appVersion + appVersion, + apiVersion, }; } diff --git a/packages/dbos-compiler/templates/dbos.create.liquid b/packages/dbos-compiler/templates/dbos.create.liquid index b710801ff..3f1293720 100644 --- a/packages/dbos-compiler/templates/dbos.create.liquid +++ b/packages/dbos-compiler/templates/dbos.create.liquid @@ -80,7 +80,35 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ } } + function $run_v2(workflow_uuid, _context, $func) { + try { + const output = $func(); + return { output }; + } catch (e) { + // -- TODO: investigate using serialize-error + return { error: { name: e.name, message: e.message, stack: e.stack } }; + } + } + + + function $require(module_name) { + if (module_name === "@dbos-inc/dbos-sdk") { + const DBOS = { + logger: { + info: (msg) => { plv8.elog(INFO, msg); }, + debug: (msg) => { plv8.elog(DEBUG1, msg); }, + warn: (msg) => { plv8.elog(WARNING, msg); }, + error: (msg) => { plv8.elog(ERROR, msg); }, + } + }; + + return { DBOS }; + } + } + globalThis.{{appVersion}}$run = $run; + globalThis.{{appVersion}}$run_v2 = $run_v2; + globalThis.require = $require; $$ LANGUAGE plv8 IMMUTABLE STRICT; diff --git a/packages/dbos-compiler/templates/method.create.liquid b/packages/dbos-compiler/templates/method.create.liquid index d410b594b..2094a5913 100644 --- a/packages/dbos-compiler/templates/method.create.liquid +++ b/packages/dbos-compiler/templates/method.create.liquid @@ -7,11 +7,14 @@ CREATE OR REPLACE FUNCTION "{{appVersion}}{{className}}_{{methodName}}_f"( if (globalThis.${{appVersion}}run === undefined) plv8.execute('SELECT dbos.{{appVersion}}run_init();'); if (globalThis.{{appVersion}}{{moduleName}} === undefined) plv8.execute('SELECT "{{appVersion}}{{moduleName}}_init"();'); - const $run = globalThis.{{appVersion}}$run; const { {{className}} } = globalThis.{{appVersion}}{{moduleName}}; - + {% if apiVersion == 1 %} + const $run = globalThis.{{appVersion}}$run; return $run(_workflow_uuid, _context, (ctx) => {{className}}.{{methodName}}(ctx, ..._params)); - + {% elsif apiVersion == 2 %} + const $run = globalThis.{{appVersion}}$run_v2; + return $run(_workflow_uuid, _context, () => {{className}}.{{methodName}}(..._params)); + {% endif %} $$ LANGUAGE plv8; CREATE OR REPLACE PROCEDURE "{{appVersion}}{{className}}_{{methodName}}_p"( diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 5bf1b4844..d19f543f6 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -26,11 +26,9 @@ describe("operations-test", () => { [config,] = parseConfigFile(); testRuntime = await createInternalTestRuntime([StoredProcTest], config) - await testRuntime.queryUserDB(` - DROP ROUTINE IF EXISTS "StoredProcTest_getGreetCountLocal_p"; - DROP ROUTINE IF EXISTS "StoredProcTest_getGreetCountLocal_f"; - DROP ROUTINE IF EXISTS "StoredProcTest_helloProcedureLocal_p"; - DROP ROUTINE IF EXISTS "StoredProcTest_helloProcedureLocal_f";`); + const localProcs = ["getGreetCountLocal", "helloProcedureLocal", "helloProcedure_v2_local"]; + const sqlDropLocalProcs = localProcs.map((proc) => `DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_p"; DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_f";`).join("\n"); + await testRuntime.queryUserDB(sqlDropLocalProcs); }); afterAll(async () => { @@ -130,7 +128,7 @@ describe("operations-test", () => { test("test-procLocalGreetingWorkflow", async () => { const wfUUID = uuidv1(); - const user = `procLocalWF_${Date.now()}`; + const user = `procLocalWF_${Date.now()}`; const res = await testRuntime.invokeWorkflow(StoredProcTest, wfUUID).procLocalGreetingWorkflow(user); expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); @@ -175,24 +173,29 @@ describe("operations-test", () => { DBOS.setConfig(config); await DBOS.launch(); - const wfUUID = uuidv1(); - const user = `txAndProcWFv2_${Date.now()}`; - const res = await DBOS.withNextWorkflowID(wfUUID, async () => { - return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); - }) + try { - expect(res.count).toBe(0); - expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + const wfUUID = uuidv1(); + const user = `txAndProcWFv2_${Date.now()}`; + const res = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); + }) - const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); - expect(txRows.length).toBe(2); - expect(txRows[0].function_id).toBe(0); - expect(txRows[0].output).toBe("0"); - expectNullResult(txRows[0].error); + expect(res.count).toBe(0); + expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - expect(txRows[1].function_id).toBe(1); - expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - expectNullResult(txRows[1].error); + const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); + expect(txRows.length).toBe(2); + expect(txRows[0].function_id).toBe(0); + expect(txRows[0].output).toBe("0"); + expectNullResult(txRows[0].error); + + expect(txRows[1].function_id).toBe(1); + expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expectNullResult(txRows[1].error); + } finally { + await DBOS.shutdown(); + } }); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 8914e61aa..a014992e2 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -121,7 +121,8 @@ export class StoredProcTest { static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; }> { // Retrieve the number of times this user has been greeted. const count = await StoredProcTest.getGreetCountTx_v2(user); - const greeting = await StoredProcTest.helloProcedure_v2_local(user); + // const greeting = await StoredProcTest.helloProcedure_v2_local(user); + const greeting = await StoredProcTest.helloProcedure_v2(user); return { count, greeting }; } @@ -142,6 +143,16 @@ export class StoredProcTest { const greet_count = rows[0].greet_count; return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; } + + @DBOS.storedProcedure() + static async helloProcedure_v2(user: string): Promise { + const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; + DBOS.logger.warn(`helloProcedure_v2: ${query}`); + // const { rows } = await DBOS.pgClient.query(query, [user]); + // const greet_count = rows[0].greet_count; + const greet_count = 1; + return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; + } } From 19ae15686e412bd7651140aa97777c7eff08b77d Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Sun, 12 Jan 2025 11:57:38 -0500 Subject: [PATCH 28/39] update run_v2 to use function.apply to have a custom this object w/ context info --- .../templates/dbos.create.liquid | 87 +++++++++++++------ .../templates/method.create.liquid | 2 +- tests/proc-test/src/operations.test.ts | 7 +- tests/proc-test/src/operations.ts | 13 ++- 4 files changed, 74 insertions(+), 35 deletions(-) diff --git a/packages/dbos-compiler/templates/dbos.create.liquid b/packages/dbos-compiler/templates/dbos.create.liquid index 3f1293720..5d4ae8ea3 100644 --- a/packages/dbos-compiler/templates/dbos.create.liquid +++ b/packages/dbos-compiler/templates/dbos.create.liquid @@ -46,6 +46,25 @@ $$ LANGUAGE plv8; CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ + function $query(queryText, values) { + const result = plv8.execute(queryText, values ?? []); + plv8.elog(DEBUG1, "DBOSQuery.query", "result", JSON.stringify(result, (k, v) => typeof v === "bigint" ? v.toString() + "n" : v)); + if (typeof result === 'number') { + return { rowCount: result }; + } else if (Array.isArray(result)) { + return { rowCount: result.length, rows: result }; + } else { + throw new Error(`unexpected result from plv8.execute ${typeof result}`); + } + }; + + const $logger = { + info: (msg) => { plv8.elog(INFO, msg); }, + debug: (msg) => { plv8.elog(DEBUG1, msg); }, + warn: (msg) => { plv8.elog(WARNING, msg); }, + error: (msg) => { plv8.elog(ERROR, msg); }, + } + function $run(workflow_uuid, _context, $func) { try { const $ctx = { @@ -54,23 +73,8 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ authenticatedUser: _context.authenticatedUser, assumedRole: _context.assumedRole, authenticatedRoles: _context.authenticatedRoles, - query: (queryText, values) => { - const result = plv8.execute(queryText, values ?? []); - plv8.elog(DEBUG1, "DBOSQuery.query", "result", JSON.stringify(result, (k, v) => typeof v === "bigint" ? v.toString() + "n" : v)); - if (typeof result === 'number') { - return { rowCount: result }; - } else if (Array.isArray(result)) { - return { rowCount: result.length, rows: result }; - } else { - throw new Error(`unexpected result from plv8.execute ${typeof result}`); - } - }, - logger: { - info: (msg) => { plv8.elog(INFO, msg); }, - debug: (msg) => { plv8.elog(DEBUG1, msg); }, - warn: (msg) => { plv8.elog(WARNING, msg); }, - error: (msg) => { plv8.elog(ERROR, msg); }, - } + query: $query, + logger: $logger, }; const output = $func($ctx); return { output }; @@ -80,9 +84,10 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ } } - function $run_v2(workflow_uuid, _context, $func) { + function $run_v2(workflow_uuid, context, func, args) { try { - const output = $func(); + const that = new $Context(workflow_uuid, context); + const output = func.apply(that, args); return { output }; } catch (e) { // -- TODO: investigate using serialize-error @@ -90,24 +95,54 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ } } - function $require(module_name) { if (module_name === "@dbos-inc/dbos-sdk") { const DBOS = { - logger: { - info: (msg) => { plv8.elog(INFO, msg); }, - debug: (msg) => { plv8.elog(DEBUG1, msg); }, - warn: (msg) => { plv8.elog(WARNING, msg); }, - error: (msg) => { plv8.elog(ERROR, msg); }, - } + getRequest: () => this.request, + get request() { + const r = this.request; + if (!r) { throw new Error("`DBOS.request` accessed from outside of HTTP requests"); } + return r; + }, + get workflowID() { return this.workflow_id; }, + get authenticatedUser() { return this.authenticatedUser ?? ""; }, + get assumedRole() { return this.assumedRole ?? ""; }, + get authenticatedRoles() { return this.authenticatedRoles ?? []; }, + logger: $logger, + pgClient: { query: $query, } }; return { DBOS }; } + + return undefined; + } + + const $Context = class { + #workflow_id; + #request; + #authenticatedUser; + #assumedRole; + #authenticatedRoles; + + constructor(workflow_uuid, context) { + this.#workflow_id = workflow_uuid; + this.#request = context.request; + this.#authenticatedUser = context.authenticatedUser; + this.#assumedRole = context.assumedRole; + this.#authenticatedRoles = context.authenticatedRoles; + } + + get workflow_id() { return this.#workflow_id; } + get request() { return this.#request; } + get authenticatedUser() { return this.#authenticatedUser; } + get assumedRole() { return this.#assumedRole; } + get authenticatedRoles() { return this.#authenticatedRoles; } } globalThis.{{appVersion}}$run = $run; globalThis.{{appVersion}}$run_v2 = $run_v2; + globalThis.$Context = $Context; globalThis.require = $require; $$ LANGUAGE plv8 IMMUTABLE STRICT; diff --git a/packages/dbos-compiler/templates/method.create.liquid b/packages/dbos-compiler/templates/method.create.liquid index 2094a5913..77d8ba35d 100644 --- a/packages/dbos-compiler/templates/method.create.liquid +++ b/packages/dbos-compiler/templates/method.create.liquid @@ -13,7 +13,7 @@ CREATE OR REPLACE FUNCTION "{{appVersion}}{{className}}_{{methodName}}_f"( return $run(_workflow_uuid, _context, (ctx) => {{className}}.{{methodName}}(ctx, ..._params)); {% elsif apiVersion == 2 %} const $run = globalThis.{{appVersion}}$run_v2; - return $run(_workflow_uuid, _context, () => {{className}}.{{methodName}}(..._params)); + return $run(_workflow_uuid, _context, {{className}}.{{methodName}}, _params); {% endif %} $$ LANGUAGE plv8; diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index d19f543f6..5afaee310 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -183,9 +183,10 @@ describe("operations-test", () => { expect(res.count).toBe(0); expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expect(res.local).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); - expect(txRows.length).toBe(2); + expect(txRows.length).toBe(3); expect(txRows[0].function_id).toBe(0); expect(txRows[0].output).toBe("0"); expectNullResult(txRows[0].error); @@ -193,6 +194,10 @@ describe("operations-test", () => { expect(txRows[1].function_id).toBe(1); expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); expectNullResult(txRows[1].error); + + expect(txRows[2].function_id).toBe(2); + expect(txRows[2].output).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); + expectNullResult(txRows[2].error); } finally { await DBOS.shutdown(); } diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index a014992e2..777053a52 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -118,13 +118,13 @@ export class StoredProcTest { } @DBOS.workflow() - static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; }> { + static async txAndProcGreetingWorkflow_v2(user: string): Promise<{ count: number; greeting: string; local: string }> { // Retrieve the number of times this user has been greeted. const count = await StoredProcTest.getGreetCountTx_v2(user); - // const greeting = await StoredProcTest.helloProcedure_v2_local(user); const greeting = await StoredProcTest.helloProcedure_v2(user); + const local = await StoredProcTest.helloProcedure_v2_local(`${user}_local`); - return { count, greeting }; + return { count, greeting, local }; } @DBOS.transaction({ readOnly: true }) @@ -147,10 +147,9 @@ export class StoredProcTest { @DBOS.storedProcedure() static async helloProcedure_v2(user: string): Promise { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; - DBOS.logger.warn(`helloProcedure_v2: ${query}`); - // const { rows } = await DBOS.pgClient.query(query, [user]); - // const greet_count = rows[0].greet_count; - const greet_count = 1; + const { rows } = await DBOS.pgClient.query(query, [user]); + const greet_count = rows[0].greet_count; + // const greet_count = 1; return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; } } From 9aff67fdfea918a9082142faa152d7040540ab8f Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Mon, 13 Jan 2025 10:23:45 -0500 Subject: [PATCH 29/39] add proc test debug config --- .vscode/launch.json | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.vscode/launch.json b/.vscode/launch.json index ae225a3d7..d99895afb 100644 --- a/.vscode/launch.json +++ b/.vscode/launch.json @@ -34,5 +34,12 @@ "type": "node-terminal", "cwd": "${workspaceFolder}/examples/hello", }, + { + "command": "npx jest --testTimeout 1000000", + "name": "Launch Proc Test", + "request": "launch", + "type": "node-terminal", + "cwd": "${workspaceFolder}/tests/proc-test", + }, ] } From 5c4acdd0ca0f598a14ffc7d3e2240e5dcedb189a Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Mon, 13 Jan 2025 11:28:46 -0500 Subject: [PATCH 30/39] increase scheduled recover tests timeouts to 20sec --- tests/scheduler/scheduler.test.ts | 2 +- tests/scheduler/scheduler_contextfree.test.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/scheduler/scheduler.test.ts b/tests/scheduler/scheduler.test.ts index cd4c448c2..eff301aff 100644 --- a/tests/scheduler/scheduler.test.ts +++ b/tests/scheduler/scheduler.test.ts @@ -164,7 +164,7 @@ describe("scheduled-wf-tests-oaoo", () => { finally { await testRuntime.destroy(); } - }, 15000); + }, 20000); }); describe("scheduled-wf-tests-when-active", () => { diff --git a/tests/scheduler/scheduler_contextfree.test.ts b/tests/scheduler/scheduler_contextfree.test.ts index af5700856..e6f17c23e 100644 --- a/tests/scheduler/scheduler_contextfree.test.ts +++ b/tests/scheduler/scheduler_contextfree.test.ts @@ -160,7 +160,7 @@ describe("cf-scheduled-wf-tests-oaoo", () => { finally { await testRuntime.destroy(); } - }, 15000); + }, 20000); }); describe("cf-scheduled-wf-tests-when-active", () => { From 8a544deb9fc92e05af7fa25518dab043aa4173d0 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Mon, 13 Jan 2025 11:43:09 -0500 Subject: [PATCH 31/39] Revert "increase scheduled recover tests timeouts to 20sec" This reverts commit 5c4acdd0ca0f598a14ffc7d3e2240e5dcedb189a. --- tests/scheduler/scheduler.test.ts | 2 +- tests/scheduler/scheduler_contextfree.test.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/scheduler/scheduler.test.ts b/tests/scheduler/scheduler.test.ts index eff301aff..cd4c448c2 100644 --- a/tests/scheduler/scheduler.test.ts +++ b/tests/scheduler/scheduler.test.ts @@ -164,7 +164,7 @@ describe("scheduled-wf-tests-oaoo", () => { finally { await testRuntime.destroy(); } - }, 20000); + }, 15000); }); describe("scheduled-wf-tests-when-active", () => { diff --git a/tests/scheduler/scheduler_contextfree.test.ts b/tests/scheduler/scheduler_contextfree.test.ts index e6f17c23e..af5700856 100644 --- a/tests/scheduler/scheduler_contextfree.test.ts +++ b/tests/scheduler/scheduler_contextfree.test.ts @@ -160,7 +160,7 @@ describe("cf-scheduled-wf-tests-oaoo", () => { finally { await testRuntime.destroy(); } - }, 20000); + }, 15000); }); describe("cf-scheduled-wf-tests-when-active", () => { From ca945c354651cf448d0ea2a9515dfc19899ba029 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Mon, 13 Jan 2025 11:53:24 -0500 Subject: [PATCH 32/39] lint fixes --- packages/dbos-compiler/compiler.ts | 2 +- packages/dbos-compiler/tests/compiler.test.ts | 2 +- .../dbos-compiler/tests/more-compiler.test.ts | 1 + src/dbos-executor.ts | 4 ++-- src/dbos.ts | 10 +--------- src/workflow.ts | 15 ++++----------- 6 files changed, 10 insertions(+), 24 deletions(-) diff --git a/packages/dbos-compiler/compiler.ts b/packages/dbos-compiler/compiler.ts index a18fd61b4..69a94729f 100644 --- a/packages/dbos-compiler/compiler.ts +++ b/packages/dbos-compiler/compiler.ts @@ -167,7 +167,7 @@ export function removeDbosMethods(file: tsm.SourceFile) { default: { const _never: never = info.kind; // eslint-disable-next-line @typescript-eslint/restrict-template-expressions - throw new Error(`Unexpected DBOS method kind: ${info}`); + throw new Error(`Unexpected DBOS method kind: ${info.kind}`); } } } diff --git a/packages/dbos-compiler/tests/compiler.test.ts b/packages/dbos-compiler/tests/compiler.test.ts index 809fa877e..3542e927c 100644 --- a/packages/dbos-compiler/tests/compiler.test.ts +++ b/packages/dbos-compiler/tests/compiler.test.ts @@ -118,7 +118,7 @@ describe("compiler", () => { const procMethods = getProcMethods(file); expect(procMethods.length).toBe(1); - const diags = checkStoredProcNames(procMethods.map(([m, v]) => m)); + const diags = checkStoredProcNames(procMethods.map(([m, _v]) => m)); expect(diags.length).toBe(1); expect(diags[0].category === tsm.DiagnosticCategory.Error); }); diff --git a/packages/dbos-compiler/tests/more-compiler.test.ts b/packages/dbos-compiler/tests/more-compiler.test.ts index 3abf0cdfe..86603c6df 100644 --- a/packages/dbos-compiler/tests/more-compiler.test.ts +++ b/packages/dbos-compiler/tests/more-compiler.test.ts @@ -44,6 +44,7 @@ describe("more compiler", () => { testDBOSDeploy: "initializer", }; + // eslint-disable-next-line @typescript-eslint/no-unsafe-assignment const expected = Object.fromEntries(Object.entries(raw_expected).flatMap(([key, value]) => { const v1 = [key, { kind: value, version: 1 }]; const v2 = [`${key}_v2`, { kind: value, version: 2 }]; diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index 8d4a20855..d0a8d1e61 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -42,7 +42,7 @@ import { import { MethodRegistrationBase, getRegisteredOperations, getOrCreateClassRegistration, MethodRegistration, getRegisteredMethodClassName, getRegisteredMethodName, getConfiguredInstance, ConfiguredInstance, getAllRegisteredClasses } from './decorators'; import { SpanStatusCode } from '@opentelemetry/api'; import knex, { Knex } from 'knex'; -import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext, runWithDBOSContext, runWithStoredProcContext } from './context'; +import { DBOSContextImpl, InitContext, runWithWorkflowContext, runWithTransactionContext, runWithStepContext, runWithStoredProcContext } from './context'; import { HandlerRegistrationBase } from './httpServer/handler'; import { WorkflowContextDebug } from './debugger/debug_workflow'; import { deserializeError, serializeError } from 'serialize-error'; @@ -1236,7 +1236,7 @@ export class DBOSExecutor implements DBOSExecutorContext { }, { isolationLevel: IsolationLevel.ReadCommitted }); await this.userDatabase.transaction(async (client: UserDatabaseClient) => { - this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); + await this.flushResultBuffer(client, wfCtx.resultBuffer, wfCtx.workflowUUID); const func = (sql: string, args: unknown[]) => this.userDatabase.queryWithClient(client, sql, ...args); await DBOSExecutor.#recordError(func, wfCtx.workflowUUID, funcId, txn_snapshot, e, (error) => this.userDatabase.isKeyConflictError(error)); }, { isolationLevel: IsolationLevel.ReadCommitted }); diff --git a/src/dbos.ts b/src/dbos.ts index 94d2c9eaa..8ec7c4a26 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -901,16 +901,8 @@ export class DBOS { registration.procConfig = config; const invokeWrapper = async function (this: This, ...rawArgs: Args): Promise { - let inst: ConfiguredInstance | undefined = undefined; - if (typeof this === 'function') { - // This is static - } - else { + if (typeof this !== 'function') { throw new Error("Stored procedures must be static"); - // inst = this as ConfiguredInstance; - // if (!("name" in inst)) { - // throw new DBOSInvalidWorkflowTransitionError(); - // } } if (DBOS.isWithinWorkflow()) { diff --git a/src/workflow.ts b/src/workflow.ts index 182b463fe..435966240 100644 --- a/src/workflow.ts +++ b/src/workflow.ts @@ -1,20 +1,13 @@ /* eslint-disable @typescript-eslint/no-explicit-any */ -import { DBOSExecutor, OperationType, dbosNull } from "./dbos-executor"; -import { transaction_outputs } from "../schemas/user_db_schema"; +import { DBOSExecutor, OperationType } from "./dbos-executor"; import { IsolationLevel, Transaction, TransactionContext } from "./transaction"; import { StepFunction, StepContext } from "./step"; -import { DBOSError, DBOSNotRegisteredError, DBOSWorkflowConflictUUIDError } from "./error"; -import { serializeError, deserializeError } from "serialize-error"; -import { DBOSJSON, sleepms } from "./utils"; import { SystemDatabase } from "./system_database"; -import { UserDatabaseClient, pgNodeIsKeyConflictError } from "./user_database"; -import { SpanStatusCode } from "@opentelemetry/api"; -import { Span } from "@opentelemetry/sdk-trace-base"; -import { HTTPRequest, DBOSContext, DBOSContextImpl, runWithDBOSContext } from './context'; +import { UserDatabaseClient } from "./user_database"; +import { HTTPRequest, DBOSContext, DBOSContextImpl } from './context'; import { ConfiguredInstance, getRegisteredOperations } from "./decorators"; -import { StoredProcedure, StoredProcedureConfig, StoredProcedureContext, StoredProcedureContextImpl } from "./procedure"; +import { StoredProcedure, StoredProcedureContext } from "./procedure"; import { InvokeFuncsInst } from "./httpServer/handler"; -import { PoolClient } from "pg"; import { WorkflowQueue } from "./wfqueue"; export type Workflow = (ctxt: WorkflowContext, ...args: T) => Promise; From 1c346fc7abfa260881a5bf190b491a8906c0cb65 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Tue, 14 Jan 2025 11:44:35 -0500 Subject: [PATCH 33/39] use global state for context info --- .../templates/dbos.create.liquid | 63 +++++++------------ .../templates/method.create.liquid | 19 +++++- tests/proc-test/src/operations.test.ts | 20 ++++++ tests/proc-test/src/operations.ts | 32 ++++++++++ 4 files changed, 89 insertions(+), 45 deletions(-) diff --git a/packages/dbos-compiler/templates/dbos.create.liquid b/packages/dbos-compiler/templates/dbos.create.liquid index 5d4ae8ea3..02ad5f151 100644 --- a/packages/dbos-compiler/templates/dbos.create.liquid +++ b/packages/dbos-compiler/templates/dbos.create.liquid @@ -65,49 +65,51 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ error: (msg) => { plv8.elog(ERROR, msg); }, } - function $run(workflow_uuid, _context, $func) { + function $serialize_error(e) { + // -- TODO: investigate using serialize-error package + return { error: { name: e.name, message: e.message, stack: e.stack } }; + } + + function $run(workflow_uuid, context, func) { try { const $ctx = { - request: _context.request, + request: context.request, workflowUUID: workflow_uuid, - authenticatedUser: _context.authenticatedUser, - assumedRole: _context.assumedRole, - authenticatedRoles: _context.authenticatedRoles, + authenticatedUser: context.authenticatedUser, + assumedRole: context.assumedRole, + authenticatedRoles: context.authenticatedRoles, query: $query, logger: $logger, }; - const output = $func($ctx); + const output = func($ctx); return { output }; } catch (e) { - // -- TODO: investigate using serialize-error - return { error: { name: e.name, message: e.message, stack: e.stack } }; + return $serialize_error(e); } } - function $run_v2(workflow_uuid, context, func, args) { + function $run_v2(func, args) { try { - const that = new $Context(workflow_uuid, context); - const output = func.apply(that, args); + const output = func(...args); return { output }; } catch (e) { - // -- TODO: investigate using serialize-error - return { error: { name: e.name, message: e.message, stack: e.stack } }; + return $serialize_error(e); } } function $require(module_name) { if (module_name === "@dbos-inc/dbos-sdk") { const DBOS = { - getRequest: () => this.request, + getRequest: () => plv8.$dbos_context.request, get request() { - const r = this.request; + const r = plv8.$dbos_context.request; if (!r) { throw new Error("`DBOS.request` accessed from outside of HTTP requests"); } return r; }, - get workflowID() { return this.workflow_id; }, - get authenticatedUser() { return this.authenticatedUser ?? ""; }, - get assumedRole() { return this.assumedRole ?? ""; }, - get authenticatedRoles() { return this.authenticatedRoles ?? []; }, + get workflowID() { return plv8.$dbos_context.workflow_id; }, + get authenticatedUser() { return plv8.$dbos_context.authenticatedUser ?? ""; }, + get assumedRole() { return plv8.$dbos_context.assumedRole ?? ""; }, + get authenticatedRoles() { return plv8.$dbos_context.authenticatedRoles ?? []; }, logger: $logger, pgClient: { query: $query, } }; @@ -118,31 +120,8 @@ CREATE OR REPLACE FUNCTION dbos.{{appVersion}}run_init() RETURNS VOID AS $$ return undefined; } - const $Context = class { - #workflow_id; - #request; - #authenticatedUser; - #assumedRole; - #authenticatedRoles; - - constructor(workflow_uuid, context) { - this.#workflow_id = workflow_uuid; - this.#request = context.request; - this.#authenticatedUser = context.authenticatedUser; - this.#assumedRole = context.assumedRole; - this.#authenticatedRoles = context.authenticatedRoles; - } - - get workflow_id() { return this.#workflow_id; } - get request() { return this.#request; } - get authenticatedUser() { return this.#authenticatedUser; } - get assumedRole() { return this.#assumedRole; } - get authenticatedRoles() { return this.#authenticatedRoles; } - } - globalThis.{{appVersion}}$run = $run; globalThis.{{appVersion}}$run_v2 = $run_v2; - globalThis.$Context = $Context; globalThis.require = $require; $$ LANGUAGE plv8 IMMUTABLE STRICT; diff --git a/packages/dbos-compiler/templates/method.create.liquid b/packages/dbos-compiler/templates/method.create.liquid index 77d8ba35d..d50023138 100644 --- a/packages/dbos-compiler/templates/method.create.liquid +++ b/packages/dbos-compiler/templates/method.create.liquid @@ -4,16 +4,29 @@ CREATE OR REPLACE FUNCTION "{{appVersion}}{{className}}_{{methodName}}_f"( _params JSONB ) RETURNS JSONB AS $$ - if (globalThis.${{appVersion}}run === undefined) plv8.execute('SELECT dbos.{{appVersion}}run_init();'); - if (globalThis.{{appVersion}}{{moduleName}} === undefined) plv8.execute('SELECT "{{appVersion}}{{moduleName}}_init"();'); + if ( + globalThis.${{appVersion}}run === undefined || + globalThis.${{appVersion}}run_v2 === undefined || + globalThis.require === undefined + ) { + plv8.execute('SELECT dbos.{{appVersion}}run_init();'); + } + + if (globalThis.{{appVersion}}{{moduleName}} === undefined) { + plv8.execute('SELECT "{{appVersion}}{{moduleName}}_init"();'); + } const { {{className}} } = globalThis.{{appVersion}}{{moduleName}}; {% if apiVersion == 1 %} const $run = globalThis.{{appVersion}}$run; return $run(_workflow_uuid, _context, (ctx) => {{className}}.{{methodName}}(ctx, ..._params)); {% elsif apiVersion == 2 %} + plv8.elog(WARNING, "$run_v2 {{className}}.{{methodName}}"); + const $run = globalThis.{{appVersion}}$run_v2; - return $run(_workflow_uuid, _context, {{className}}.{{methodName}}, _params); + plv8.$dbos_context = {..._context, workflow_id: _workflow_uuid }; + return $run({{className}}.{{methodName}}, _params); + delete plv8.$dbos_context; {% endif %} $$ LANGUAGE plv8; diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 5afaee310..14f860d4f 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -168,6 +168,26 @@ describe("operations-test", () => { }); + test("test-v2proc-context", async () => { + DBOS.setConfig(config); + await DBOS.launch(); + + try { + + + const wfUUID = uuidv1(); + console.log(`wfUUID ${wfUUID}`); + const result = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.getWorkflowID(); + }); + + expect(result).toBe(wfUUID); + } finally { + await DBOS.shutdown(); + } + + }); + test("test-txAndProcGreetingWorkflow_v2", async () => { DBOS.setConfig(config); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 777053a52..5e3658b76 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -152,6 +152,38 @@ export class StoredProcTest { // const greet_count = 1; return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; } + + @DBOS.workflow() + static async getWorkflowID_workflow(): Promise { + DBOS.logger.info(`getWorkflowID_workflow ${DBOS.workflowID}`); + return StoredProcTest.getWorkflowID(); + } + + @DBOS.storedProcedure() + static async getWorkflowID(): Promise { + DBOS.logger.info(`getWorkflowID ${DBOS.workflowID}`); + return DBOS.workflowID; + } + + @DBOS.storedProcedure() + static async getAssumedRole(): Promise { + return DBOS.assumedRole; + } + + @DBOS.storedProcedure() + static async getAuthenticatedUser(): Promise { + return DBOS.authenticatedUser; + } + + @DBOS.storedProcedure() + static async getAuthenticatedRoles(): Promise { + return DBOS.authenticatedRoles; + } + + + + + } From ea1ec3367674e72b878934a861606b42de7a2073 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Tue, 14 Jan 2025 14:39:56 -0500 Subject: [PATCH 34/39] fix test --- tests/proc-test/src/operations.test.ts | 5 +---- tests/proc-test/src/operations.ts | 5 ++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index 14f860d4f..ef4bb301d 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -173,12 +173,9 @@ describe("operations-test", () => { await DBOS.launch(); try { - - const wfUUID = uuidv1(); - console.log(`wfUUID ${wfUUID}`); const result = await DBOS.withNextWorkflowID(wfUUID, async () => { - return await StoredProcTest.getWorkflowID(); + return await StoredProcTest.getWorkflowContext(); }); expect(result).toBe(wfUUID); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index 5e3658b76..d8facefe9 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -154,17 +154,16 @@ export class StoredProcTest { } @DBOS.workflow() - static async getWorkflowID_workflow(): Promise { - DBOS.logger.info(`getWorkflowID_workflow ${DBOS.workflowID}`); + static async getWorkflowContext(): Promise { return StoredProcTest.getWorkflowID(); } @DBOS.storedProcedure() static async getWorkflowID(): Promise { - DBOS.logger.info(`getWorkflowID ${DBOS.workflowID}`); return DBOS.workflowID; } + @DBOS.storedProcedure() static async getAssumedRole(): Promise { return DBOS.assumedRole; From 70efafeb303734fed5a08875c7541f3d9e4941e5 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 15 Jan 2025 14:25:00 -0800 Subject: [PATCH 35/39] update stored proc invoke wrapper to match tx and step --- src/dbos.ts | 31 ++++++++++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/src/dbos.ts b/src/dbos.ts index 78dac5c02..5024eeb20 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -927,8 +927,37 @@ export class DBOS { registration.registeredFunction as unknown as StoredProcedure, wfctx, ...rawArgs); } + const wfId = getNextWFID(undefined); + + const pctx = getCurrentContextStore(); + let span = pctx?.span; + if (!span) { + span = DBOS.executor.tracer.startSpan( + pctx?.operationCaller || "transactionCaller", + { + operationType: pctx?.operationType, + authenticatedUser: pctx?.authenticatedUser, + assumedRole: pctx?.assumedRole, + authenticatedRoles: pctx?.authenticatedRoles, + }, + ); + } + + let parentCtx: DBOSContextImpl | undefined = undefined; + if (pctx) { + parentCtx = pctx.ctx as DBOSContextImpl + } + if (!parentCtx) { + parentCtx = new DBOSContextImpl(pctx?.operationCaller || "workflowCaller", span, DBOS.logger as GlobalLogger); + parentCtx.request = pctx?.request || {}; + parentCtx.authenticatedUser = pctx?.authenticatedUser || ""; + parentCtx.assumedRole = pctx?.assumedRole || ""; + parentCtx.authenticatedRoles = pctx?.authenticatedRoles || []; + } + const wfParams: WorkflowParams = { - // configuredInstance: inst + parentCtx, + workflowUUID: wfId, }; return await DBOS.executor.procedure( From 2db2d7b213b6c5ee61710f1bdb0e33aed3e9121e Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 15 Jan 2025 14:25:43 -0800 Subject: [PATCH 36/39] cleanup --- .../templates/method.create.liquid | 2 -- tests/proc-test/src/operations.ts | 35 ++++++------------- 2 files changed, 10 insertions(+), 27 deletions(-) diff --git a/packages/dbos-compiler/templates/method.create.liquid b/packages/dbos-compiler/templates/method.create.liquid index d50023138..d5a9aa192 100644 --- a/packages/dbos-compiler/templates/method.create.liquid +++ b/packages/dbos-compiler/templates/method.create.liquid @@ -21,8 +21,6 @@ CREATE OR REPLACE FUNCTION "{{appVersion}}{{className}}_{{methodName}}_f"( const $run = globalThis.{{appVersion}}$run; return $run(_workflow_uuid, _context, (ctx) => {{className}}.{{methodName}}(ctx, ..._params)); {% elsif apiVersion == 2 %} - plv8.elog(WARNING, "$run_v2 {{className}}.{{methodName}}"); - const $run = globalThis.{{appVersion}}$run_v2; plv8.$dbos_context = {..._context, workflow_id: _workflow_uuid }; return $run({{className}}.{{methodName}}, _params); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index d8facefe9..bf1156fc6 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -53,8 +53,6 @@ export class StoredProcTest { return greeting; } - - @StoredProcedure({ readOnly: true, executeLocally: true }) static async getGreetCountLocal(ctxt: StoredProcedureContext, user: string): Promise { const query = "SELECT greet_count FROM dbos_hello WHERE name = $1;"; @@ -78,7 +76,6 @@ export class StoredProcTest { return parseInt(rows[0].count); } - @Workflow() static async procLocalGreetingWorkflow(ctxt: WorkflowContext, user: string): Promise<{ count: number; greeting: string; rowCount: number }> { const count = await ctxt.invoke(StoredProcTest).getGreetCountLocal(user); @@ -87,8 +84,6 @@ export class StoredProcTest { return { count, greeting, rowCount }; } - - @Transaction({ readOnly: true }) static async getGreetCountTx(ctxt: TransactionContext, user: string): Promise { const query = "SELECT greet_count FROM dbos_hello WHERE name = ?;"; @@ -154,35 +149,25 @@ export class StoredProcTest { } @DBOS.workflow() - static async getWorkflowContext(): Promise { - return StoredProcTest.getWorkflowID(); + static async wf_GetWorkflowID() { + return StoredProcTest.sp_GetWorkflowID(); } @DBOS.storedProcedure() - static async getWorkflowID(): Promise { + static async sp_GetWorkflowID() { return DBOS.workflowID; } - @DBOS.storedProcedure() - static async getAssumedRole(): Promise { - return DBOS.assumedRole; + static async sp_GetAuth() { + return { + user: DBOS.authenticatedUser, + roles: DBOS.authenticatedRoles, + }; } @DBOS.storedProcedure() - static async getAuthenticatedUser(): Promise { - return DBOS.authenticatedUser; + static async sp_GetRequest() { + return DBOS.request; } - - @DBOS.storedProcedure() - static async getAuthenticatedRoles(): Promise { - return DBOS.authenticatedRoles; - } - - - - - } - - From 63e26991006a76bec8b918c31ca884879ca527c2 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 15 Jan 2025 14:26:03 -0800 Subject: [PATCH 37/39] move v2 proc tests to separate describe block --- tests/proc-test/src/operations.test.ts | 161 ++++++++++++++++--------- 1 file changed, 104 insertions(+), 57 deletions(-) diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index ef4bb301d..f1ef7cd07 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -7,6 +7,7 @@ import { transaction_outputs } from "@dbos-inc/dbos-sdk/schemas/user_db_schema"; import { TestingRuntimeImpl, createInternalTestRuntime } from "@dbos-inc/dbos-sdk/dist/src/testing/testing_runtime"; import { DBOSConfig } from "@dbos-inc/dbos-sdk"; import { Client, ClientConfig } from "pg"; +import { runWithTopContext } from "../../../dist/src/context"; async function runSql(config: ClientConfig, func: (client: Client) => Promise) { const client = new Client(config); @@ -18,17 +19,115 @@ async function runSql(config: ClientConfig, func: (client: Client) => Promise } } -describe("operations-test", () => { +async function dropLocalProcs(testRuntime: TestingRuntime) { + const localProcs = ["getGreetCountLocal", "helloProcedureLocal", "helloProcedure_v2_local"]; + const sqlDropLocalProcs = localProcs.map((proc) => `DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_p"; DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_f";`).join("\n"); + await testRuntime.queryUserDB(sqlDropLocalProcs); +} + +describe("stored-proc-v2-test", () => { + let config: DBOSConfig; + + beforeAll(async () => { + [config,] = parseConfigFile(); + const testRuntime = await createInternalTestRuntime([StoredProcTest], config) + try { + await dropLocalProcs(testRuntime); + await testRuntime.destroy(); + } finally { + testRuntime.destroy(); + } + }); + + beforeEach(async () => { + DBOS.setConfig(config); + await DBOS.launch(); + }); + + afterEach(async () => { + await DBOS.shutdown(); + }); + + test("wf_GetWorkflowID", async () => { + const wfUUID = `wf-${Date.now()}`; + const result = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.wf_GetWorkflowID(); + }); + + expect(result).toBe(wfUUID); + }); + + test("sp_GetWorkflowID", async () => { + const wfUUID = `sp-${Date.now()}`; + const result = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.sp_GetWorkflowID(); + }); + + expect(result).toBe(wfUUID); + }); + + test("sp_GetAuth", async () => { + const now = `${Date.now()}`; + const user = `user-${now}`; + const roles = [`role-1-${now}`, `role-2-${now}`, `role-3-${now}`]; + const actual = await DBOS.withAuthedContext(user, roles, async () => { + return await StoredProcTest.sp_GetAuth(); + }); + expect(actual).toEqual({ user, roles }); + }); + + test("sp_GetRequest", async () => { + const ctx = { + request: { requestID: `requestID-${Date.now()}` } + }; + const actual = await runWithTopContext(ctx, async () => { + return await StoredProcTest.sp_GetRequest(); + }); + expect(actual.requestID).toEqual(ctx.request.requestID); + }); + + test("test-txAndProcGreetingWorkflow_v2", async () => { + + const wfUUID = uuidv1(); + const user = `txAndProcWFv2_${Date.now()}`; + const res = await DBOS.withNextWorkflowID(wfUUID, async () => { + return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); + }) + + expect(res.count).toBe(0); + expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expect(res.local).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); + + const dbClient = new Client(config.poolConfig); + try { + await dbClient.connect(); + const { rows: txRows } = await dbClient.query("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", [wfUUID]); + expect(txRows.length).toBe(3); + expect(txRows[0].function_id).toBe(0); + expect(txRows[0].output).toBe("0"); + expectNullResult(txRows[0].error); + + expect(txRows[1].function_id).toBe(1); + expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); + expectNullResult(txRows[1].error); + + expect(txRows[2].function_id).toBe(2); + expect(txRows[2].output).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); + expectNullResult(txRows[2].error); + } finally { + await dbClient.end(); + } + }); +}); + +describe("stored-proc-test", () => { let config: DBOSConfig; let testRuntime: TestingRuntime; beforeAll(async () => { [config,] = parseConfigFile(); testRuntime = await createInternalTestRuntime([StoredProcTest], config) - - const localProcs = ["getGreetCountLocal", "helloProcedureLocal", "helloProcedure_v2_local"]; - const sqlDropLocalProcs = localProcs.map((proc) => `DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_p"; DROP ROUTINE IF EXISTS "StoredProcTest_${proc}_f";`).join("\n"); - await testRuntime.queryUserDB(sqlDropLocalProcs); + await dropLocalProcs(testRuntime); }); afterAll(async () => { @@ -168,58 +267,6 @@ describe("operations-test", () => { }); - test("test-v2proc-context", async () => { - DBOS.setConfig(config); - await DBOS.launch(); - - try { - const wfUUID = uuidv1(); - const result = await DBOS.withNextWorkflowID(wfUUID, async () => { - return await StoredProcTest.getWorkflowContext(); - }); - - expect(result).toBe(wfUUID); - } finally { - await DBOS.shutdown(); - } - - }); - - test("test-txAndProcGreetingWorkflow_v2", async () => { - - DBOS.setConfig(config); - await DBOS.launch(); - - try { - - const wfUUID = uuidv1(); - const user = `txAndProcWFv2_${Date.now()}`; - const res = await DBOS.withNextWorkflowID(wfUUID, async () => { - return await StoredProcTest.txAndProcGreetingWorkflow_v2(user); - }) - - expect(res.count).toBe(0); - expect(res.greeting).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - expect(res.local).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); - - const txRows = await testRuntime.queryUserDB("SELECT * FROM dbos.transaction_outputs WHERE workflow_uuid=$1", wfUUID); - expect(txRows.length).toBe(3); - expect(txRows[0].function_id).toBe(0); - expect(txRows[0].output).toBe("0"); - expectNullResult(txRows[0].error); - - expect(txRows[1].function_id).toBe(1); - expect(txRows[1].output).toMatch(`Hello, ${user}! You have been greeted 1 times.`); - expectNullResult(txRows[1].error); - - expect(txRows[2].function_id).toBe(2); - expect(txRows[2].output).toMatch(`Hello, ${user}_local! You have been greeted 1 times.`); - expectNullResult(txRows[2].error); - } finally { - await DBOS.shutdown(); - } - }); - test("test-procErrorWorkflow", async () => { const wfid = uuidv1(); From d53561e37037ece7dc5538b991453e0914e966fd Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 15 Jan 2025 14:31:23 -0800 Subject: [PATCH 38/39] linter --- tests/proc-test/src/operations.test.ts | 3 +-- tests/proc-test/src/operations.ts | 5 ++++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/proc-test/src/operations.test.ts b/tests/proc-test/src/operations.test.ts index f1ef7cd07..d7c733b52 100644 --- a/tests/proc-test/src/operations.test.ts +++ b/tests/proc-test/src/operations.test.ts @@ -33,9 +33,8 @@ describe("stored-proc-v2-test", () => { const testRuntime = await createInternalTestRuntime([StoredProcTest], config) try { await dropLocalProcs(testRuntime); - await testRuntime.destroy(); } finally { - testRuntime.destroy(); + await testRuntime.destroy(); } }); diff --git a/tests/proc-test/src/operations.ts b/tests/proc-test/src/operations.ts index bf1156fc6..afc3063aa 100644 --- a/tests/proc-test/src/operations.ts +++ b/tests/proc-test/src/operations.ts @@ -144,7 +144,6 @@ export class StoredProcTest { const query = "INSERT INTO dbos_hello (name, greet_count) VALUES ($1, 1) ON CONFLICT (name) DO UPDATE SET greet_count = dbos_hello.greet_count + 1 RETURNING greet_count;"; const { rows } = await DBOS.pgClient.query(query, [user]); const greet_count = rows[0].greet_count; - // const greet_count = 1; return `Hello, ${user}! You have been greeted ${greet_count} times.\n`; } @@ -153,6 +152,8 @@ export class StoredProcTest { return StoredProcTest.sp_GetWorkflowID(); } + /* eslint-disable @typescript-eslint/require-await */ + @DBOS.storedProcedure() static async sp_GetWorkflowID() { return DBOS.workflowID; @@ -170,4 +171,6 @@ export class StoredProcTest { static async sp_GetRequest() { return DBOS.request; } + + /* eslint-enable @typescript-eslint/require-await */ } From c7a05c3bf98916d13f2d9c1e71cc584eacf0fb93 Mon Sep 17 00:00:00 2001 From: Harry Pierson Date: Wed, 15 Jan 2025 16:52:24 -0800 Subject: [PATCH 39/39] PR Feedback --- src/dbos-executor.ts | 4 +--- src/dbos.ts | 9 ++++++--- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/dbos-executor.ts b/src/dbos-executor.ts index e46606ab2..1db36f321 100644 --- a/src/dbos-executor.ts +++ b/src/dbos-executor.ts @@ -38,7 +38,6 @@ import { DrizzleUserDatabase, UserDatabaseClient, pgNodeIsKeyConflictError, - createDBIfDoesNotExist, } from './user_database'; import { MethodRegistrationBase, getRegisteredOperations, getOrCreateClassRegistration, MethodRegistration, getRegisteredMethodClassName, getRegisteredMethodName, getConfiguredInstance, ConfiguredInstance, getAllRegisteredClasses } from './decorators'; import { SpanStatusCode } from '@opentelemetry/api'; @@ -380,7 +379,6 @@ export class DBOSExecutor implements DBOSExecutorContext { this.logger.debug(`Loaded ${length} ORM entities`); } - await(createDBIfDoesNotExist(this.config.poolConfig, this.logger)) this.configureDbClient(); if (!this.userDatabase) { @@ -660,7 +658,7 @@ export class DBOSExecutor implements DBOSExecutorContext { // Synchronously set the workflow's status to PENDING and record workflow inputs (for non single-transaction workflows). // We have to do it for all types of workflows because operation_outputs table has a foreign key constraint on workflow status table. if ((wCtxt.tempWfOperationType !== TempWorkflowType.transaction - && wCtxt.tempWfOperationType !== TempWorkflowType.procedure) + && wCtxt.tempWfOperationType !== TempWorkflowType.procedure) || params.queueName !== undefined ) { // TODO: Make this transactional (and with the queue step below) diff --git a/src/dbos.ts b/src/dbos.ts index 5024eeb20..bbe9d5cb5 100644 --- a/src/dbos.ts +++ b/src/dbos.ts @@ -827,7 +827,8 @@ export class DBOS { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise> + ) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.txnConfig = config; @@ -912,7 +913,8 @@ export class DBOS { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise> + ) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.procConfig = config; @@ -983,7 +985,8 @@ export class DBOS { function decorator( target: object, propertyKey: string, - inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise>) { + inDescriptor: TypedPropertyDescriptor<(this: This, ...args: Args) => Promise> + ) { const { descriptor, registration } = registerAndWrapDBOSFunction(target, propertyKey, inDescriptor); registration.commConfig = config;