From ae0c7bc37e928a1377188ed4f68289af2ad62314 Mon Sep 17 00:00:00 2001 From: Chris <143021053+kunwp1@users.noreply.github.com> Date: Mon, 13 Jan 2025 14:21:39 -0800 Subject: [PATCH] Normalize and Improve Operator Runtime Statistics Handling (#3171) This PR addresses schema normalization and logic improvements for tracking operator runtime statistics in a workflow execution system. It introduces changes to the database schema, migration scripts, and Scala code responsible for inserting and managing runtime statistics. The goal is to reduce redundancy, improve maintainability, and ensure data consistency between `operator_executions` and `operator_runtime_statistics`. ### Schema Design 1. New Table Design: - `operator_executions`: Tracks execution metadata for each operator in a workflow execution. Each row contains `operator_execution_id`, `workflow_execution_id`, and `operator_id`. This table ensures that operator executions are uniquely identifiable. - `operator_runtime_statistics`: Tracks runtime statistics for each operator execution at specific timestamps. It includes `operator_execution_id` as a foreign key, ensuring a direct reference to `operator_executions`. 2. Normalization Improvements: - Replaced repeated `execution_id` and `operator_id` in `workflow_runtime_statistics` with a single foreign key `operator_execution_id`, pointing to `operator_executions`. - Split the previous large `workflow_runtime_statistics` table into smaller, more manageable tables, eliminating redundancy and improving data integrity. 3. Indexes and Keys: - Added a composite index on `operator_execution_id` and `time` in `operator_runtime_statistics` to speed up joins and queries ordered by time. ### Testing The `core/scripts/sql/update/19.sql` will create the two new tables, `operator_executions` and `operator_runtime_statistics`, and migrate the data from `workflow_runtime_statistics` to those two tables. --------- Co-authored-by: Kunwoo Park Co-authored-by: Kunwoo Park Co-authored-by: Kunwoo Park Co-authored-by: Kunwoo Park Co-authored-by: Kunwoo Park Co-authored-by: Kunwoo Park --- .../scheduling/CostEstimator.scala | 65 ++- .../workflow/WorkflowExecutionsResource.scala | 80 ++- .../web/service/ExecutionStatsService.scala | 44 +- .../service/WorkflowExecutionService.scala | 30 +- .../scheduling/DefaultCostEstimatorSpec.scala | 149 +++-- .../texera/dao/jooq/generated/Indexes.java | 13 +- .../ics/texera/dao/jooq/generated/Keys.java | 25 +- .../ics/texera/dao/jooq/generated/Tables.java | 18 +- .../texera/dao/jooq/generated/TexeraDb.java | 23 +- .../generated/tables/OperatorExecutions.java | 169 ++++++ .../tables/OperatorRuntimeStatistics.java | 194 +++++++ .../tables/WorkflowRuntimeStatistics.java | 208 ------- .../tables/daos/OperatorExecutionsDao.java | 91 +++ .../daos/OperatorRuntimeStatisticsDao.java | 170 ++++++ .../daos/WorkflowRuntimeStatisticsDao.java | 198 ------- .../interfaces/IOperatorExecutions.java | 62 ++ .../IOperatorRuntimeStatistics.java | 123 ++++ .../IWorkflowRuntimeStatistics.java | 143 ----- .../tables/pojos/OperatorExecutions.java | 101 ++++ ...cs.java => OperatorRuntimeStatistics.java} | 86 +-- .../records/OperatorExecutionsRecord.java | 207 +++++++ .../OperatorRuntimeStatisticsRecord.java | 449 +++++++++++++++ .../WorkflowRuntimeStatisticsRecord.java | 529 ------------------ .../workflow-runtime-statistics.component.ts | 10 +- .../workflow-executions.service.ts | 6 +- ...tics.ts => workflow-runtime-statistics.ts} | 2 +- core/scripts/sql/texera_ddl.sql | 45 +- core/scripts/sql/update/19.sql | 55 ++ 28 files changed, 1983 insertions(+), 1312 deletions(-) create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorExecutions.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorRuntimeStatistics.java delete mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/WorkflowRuntimeStatistics.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorExecutionsDao.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorRuntimeStatisticsDao.java delete mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/WorkflowRuntimeStatisticsDao.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorExecutions.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorRuntimeStatistics.java delete mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IWorkflowRuntimeStatistics.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorExecutions.java rename core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/{WorkflowRuntimeStatistics.java => OperatorRuntimeStatistics.java} (66%) create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorExecutionsRecord.java create mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorRuntimeStatisticsRecord.java delete mode 100644 core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/WorkflowRuntimeStatisticsRecord.java rename core/gui/src/app/dashboard/type/{operator-runtime-statistics.ts => workflow-runtime-statistics.ts} (84%) create mode 100644 core/scripts/sql/update/19.sql diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostEstimator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostEstimator.scala index c675d44f154..60523145ca1 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostEstimator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/CostEstimator.scala @@ -8,11 +8,12 @@ import edu.uci.ics.amber.core.virtualidentity.ActorVirtualIdentity import edu.uci.ics.texera.dao.SqlServer import edu.uci.ics.texera.dao.SqlServer.withTransaction import edu.uci.ics.texera.dao.jooq.generated.Tables.{ + OPERATOR_EXECUTIONS, + OPERATOR_RUNTIME_STATISTICS, WORKFLOW_EXECUTIONS, - WORKFLOW_RUNTIME_STATISTICS, WORKFLOW_VERSION } -import edu.uci.ics.texera.dao.jooq.generated.tables.pojos.WorkflowRuntimeStatistics +import edu.uci.ics.texera.web.resource.dashboard.user.workflow.WorkflowExecutionsResource.WorkflowRuntimeStatistics import org.jooq.types.UInteger import scala.jdk.CollectionConverters.ListHasAsScala @@ -99,36 +100,38 @@ class DefaultCostEstimator( val widAsUInteger = UInteger.valueOf(wid) val rawStats = context .select( - WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, - WORKFLOW_RUNTIME_STATISTICS.TIME, - WORKFLOW_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, - WORKFLOW_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, - WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID + OPERATOR_EXECUTIONS.OPERATOR_ID, + OPERATOR_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, + OPERATOR_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, + OPERATOR_RUNTIME_STATISTICS.TIME, + OPERATOR_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, + OPERATOR_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, + OPERATOR_RUNTIME_STATISTICS.IDLE_TIME, + OPERATOR_RUNTIME_STATISTICS.NUM_WORKERS + ) + .from(OPERATOR_EXECUTIONS) + .join(OPERATOR_RUNTIME_STATISTICS) + .on( + OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID.eq( + OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID + ) ) - .from(WORKFLOW_RUNTIME_STATISTICS) .where( - WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID - .eq(widAsUInteger) - .and( - WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID.eq( - context - .select( - WORKFLOW_EXECUTIONS.EID - ) - .from(WORKFLOW_EXECUTIONS) - .join(WORKFLOW_VERSION) - .on(WORKFLOW_VERSION.VID.eq(WORKFLOW_EXECUTIONS.VID)) - .where( - WORKFLOW_VERSION.WID - .eq(widAsUInteger) - .and(WORKFLOW_EXECUTIONS.STATUS.eq(3.toByte)) - ) - .orderBy(WORKFLOW_EXECUTIONS.STARTING_TIME.desc()) - .limit(1) + OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID.eq( + context + .select(WORKFLOW_EXECUTIONS.EID) + .from(WORKFLOW_EXECUTIONS) + .join(WORKFLOW_VERSION) + .on(WORKFLOW_VERSION.VID.eq(WORKFLOW_EXECUTIONS.VID)) + .where( + WORKFLOW_VERSION.WID + .eq(widAsUInteger) + .and(WORKFLOW_EXECUTIONS.STATUS.eq(3.toByte)) ) - ) + .orderBy(WORKFLOW_EXECUTIONS.STARTING_TIME.desc()) + .limit(1) + ) ) - .orderBy(WORKFLOW_RUNTIME_STATISTICS.TIME, WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID) .fetchInto(classOf[WorkflowRuntimeStatistics]) .asScala .toList @@ -136,9 +139,9 @@ class DefaultCostEstimator( None } else { val cumulatedStats = rawStats.foldLeft(Map.empty[String, Double]) { (acc, stat) => - val opTotalExecutionTime = acc.getOrElse(stat.getOperatorId, 0.0) - acc + (stat.getOperatorId -> (opTotalExecutionTime + (stat.getDataProcessingTime - .doubleValue() + stat.getControlProcessingTime.doubleValue()) / 1e9)) + val opTotalExecutionTime = acc.getOrElse(stat.operatorId, 0.0) + acc + (stat.operatorId -> (opTotalExecutionTime + (stat.dataProcessingTime + .doubleValue() + stat.controlProcessingTime.doubleValue()) / 1e9)) } Some(cumulatedStats) } diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala index d3311ea4030..ab6cec64c59 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/dashboard/user/workflow/WorkflowExecutionsResource.scala @@ -9,16 +9,19 @@ import edu.uci.ics.texera.web.auth.SessionUser import edu.uci.ics.texera.dao.jooq.generated.Tables.{ USER, WORKFLOW_EXECUTIONS, - WORKFLOW_RUNTIME_STATISTICS, + OPERATOR_EXECUTIONS, + OPERATOR_RUNTIME_STATISTICS, WORKFLOW_VERSION } import edu.uci.ics.texera.dao.jooq.generated.tables.daos.{ - WorkflowExecutionsDao, - WorkflowRuntimeStatisticsDao + OperatorExecutionsDao, + OperatorRuntimeStatisticsDao, + WorkflowExecutionsDao } import edu.uci.ics.texera.dao.jooq.generated.tables.pojos.{ WorkflowExecutions, - WorkflowRuntimeStatistics + OperatorExecutions, + OperatorRuntimeStatistics } import edu.uci.ics.texera.web.resource.dashboard.user.workflow.WorkflowExecutionsResource._ import edu.uci.ics.texera.web.service.ExecutionsMetadataPersistService @@ -40,7 +43,10 @@ object WorkflowExecutionsResource { .getInstance(StorageConfig.jdbcUrl, StorageConfig.jdbcUsername, StorageConfig.jdbcPassword) .createDSLContext() final private lazy val executionsDao = new WorkflowExecutionsDao(context.configuration) - final private lazy val workflowRuntimeStatisticsDao = new WorkflowRuntimeStatisticsDao( + final private lazy val operatorExecutionsDao = new OperatorExecutionsDao( + context.configuration + ) + final private lazy val operatorRuntimeStatisticsDao = new OperatorRuntimeStatisticsDao( context.configuration ) @@ -87,8 +93,19 @@ object WorkflowExecutionsResource { } } - def insertWorkflowRuntimeStatistics(list: util.ArrayList[WorkflowRuntimeStatistics]): Unit = { - workflowRuntimeStatisticsDao.insert(list); + def insertOperatorExecutions( + list: util.ArrayList[OperatorExecutions] + ): util.HashMap[String, ULong] = { + operatorExecutionsDao.insert(list); + val result = new util.HashMap[String, ULong]() + list.forEach(execution => { + result.put(execution.getOperatorId, execution.getOperatorExecutionId) + }) + result + } + + def insertOperatorRuntimeStatistics(list: util.ArrayList[OperatorRuntimeStatistics]): Unit = { + operatorRuntimeStatisticsDao.insert(list); } case class WorkflowExecutionEntry( @@ -105,15 +122,10 @@ object WorkflowExecutionsResource { logLocation: String ) - case class ExecutionResultEntry( - eId: UInteger, - result: String - ) - - case class OperatorRuntimeStatistics( + case class WorkflowRuntimeStatistics( operatorId: String, - inputTupleCount: UInteger, - outputTupleCount: UInteger, + inputTupleCount: ULong, + outputTupleCount: ULong, timestamp: Timestamp, dataProcessingTime: ULong, controlProcessingTime: ULong, @@ -222,26 +234,36 @@ class WorkflowExecutionsResource { def retrieveWorkflowRuntimeStatistics( @PathParam("wid") wid: UInteger, @PathParam("eid") eid: UInteger - ): List[OperatorRuntimeStatistics] = { + ): List[WorkflowRuntimeStatistics] = { context .select( - WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, - WORKFLOW_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, - WORKFLOW_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, - WORKFLOW_RUNTIME_STATISTICS.TIME, - WORKFLOW_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, - WORKFLOW_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, - WORKFLOW_RUNTIME_STATISTICS.IDLE_TIME, - WORKFLOW_RUNTIME_STATISTICS.NUM_WORKERS + OPERATOR_EXECUTIONS.OPERATOR_ID, + OPERATOR_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, + OPERATOR_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, + OPERATOR_RUNTIME_STATISTICS.TIME, + OPERATOR_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, + OPERATOR_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, + OPERATOR_RUNTIME_STATISTICS.IDLE_TIME, + OPERATOR_RUNTIME_STATISTICS.NUM_WORKERS + ) + .from(OPERATOR_RUNTIME_STATISTICS) + .join(OPERATOR_EXECUTIONS) + .on( + OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID.eq( + OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID + ) ) - .from(WORKFLOW_RUNTIME_STATISTICS) + .join(WORKFLOW_EXECUTIONS) + .on(OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID.eq(WORKFLOW_EXECUTIONS.EID)) + .join(WORKFLOW_VERSION) + .on(WORKFLOW_EXECUTIONS.VID.eq(WORKFLOW_VERSION.VID)) .where( - WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID + WORKFLOW_VERSION.WID .eq(wid) - .and(WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID.eq(eid)) + .and(WORKFLOW_EXECUTIONS.EID.eq(eid)) ) - .orderBy(WORKFLOW_RUNTIME_STATISTICS.TIME, WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID) - .fetchInto(classOf[OperatorRuntimeStatistics]) + .orderBy(OPERATOR_RUNTIME_STATISTICS.TIME, OPERATOR_EXECUTIONS.OPERATOR_ID) + .fetchInto(classOf[WorkflowRuntimeStatistics]) .asScala .toList } diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala index a333d16d060..8494dfb2238 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala @@ -2,7 +2,6 @@ package edu.uci.ics.texera.web.service import com.google.protobuf.timestamp.Timestamp import com.typesafe.scalalogging.LazyLogging -import edu.uci.ics.amber.core.workflow.WorkflowContext import edu.uci.ics.amber.engine.architecture.controller.{ ExecutionStatsUpdate, FatalError, @@ -24,7 +23,7 @@ import edu.uci.ics.amber.error.ErrorUtils.{getOperatorFromActorIdOpt, getStackTr import edu.uci.ics.amber.core.workflowruntimestate.FatalErrorType.EXECUTION_FAILURE import edu.uci.ics.amber.core.workflowruntimestate.WorkflowFatalError import edu.uci.ics.texera.web.SubscriptionManager -import edu.uci.ics.texera.dao.jooq.generated.tables.pojos.WorkflowRuntimeStatistics +import edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorRuntimeStatistics import edu.uci.ics.texera.web.model.websocket.event.{ ExecutionDurationUpdateEvent, OperatorAggregatedMetrics, @@ -43,7 +42,7 @@ import java.util.concurrent.Executors class ExecutionStatsService( client: AmberClient, stateStore: ExecutionStateStore, - workflowContext: WorkflowContext + operatorIdToExecutionId: Map[String, ULong] ) extends SubscriptionManager with LazyLogging { private val metricsPersistThread = Executors.newSingleThreadExecutor() @@ -201,31 +200,34 @@ class ExecutionStatsService( private def storeRuntimeStatistics( operatorStatistics: scala.collection.immutable.Map[String, OperatorMetrics] ): Unit = { - // Add a try-catch to not produce an error when "workflow_runtime_statistics" table does not exist in MySQL try { - val list: util.ArrayList[WorkflowRuntimeStatistics] = - new util.ArrayList[WorkflowRuntimeStatistics]() + val runtimeStatsList: util.ArrayList[OperatorRuntimeStatistics] = + new util.ArrayList[OperatorRuntimeStatistics]() + for ((operatorId, stat) <- operatorStatistics) { - val execution = new WorkflowRuntimeStatistics() - execution.setWorkflowId(UInteger.valueOf(workflowContext.workflowId.id)) - execution.setExecutionId(UInteger.valueOf(workflowContext.executionId.id)) - execution.setOperatorId(operatorId) - execution.setInputTupleCnt( - UInteger.valueOf(stat.operatorStatistics.inputCount.map(_.tupleCount).sum) + // Create and populate the operator runtime statistics entry + val runtimeStats = new OperatorRuntimeStatistics() + runtimeStats.setOperatorExecutionId(operatorIdToExecutionId(operatorId)) + runtimeStats.setInputTupleCnt( + ULong.valueOf(stat.operatorStatistics.inputCount.map(_.tupleCount).sum) + ) + runtimeStats.setOutputTupleCnt( + ULong.valueOf(stat.operatorStatistics.outputCount.map(_.tupleCount).sum) ) - execution.setOutputTupleCnt( - UInteger.valueOf(stat.operatorStatistics.outputCount.map(_.tupleCount).sum) + runtimeStats.setStatus(maptoStatusCode(stat.operatorState)) + runtimeStats.setDataProcessingTime( + ULong.valueOf(stat.operatorStatistics.dataProcessingTime) ) - execution.setStatus(maptoStatusCode(stat.operatorState)) - execution.setDataProcessingTime(ULong.valueOf(stat.operatorStatistics.dataProcessingTime)) - execution.setControlProcessingTime( + runtimeStats.setControlProcessingTime( ULong.valueOf(stat.operatorStatistics.controlProcessingTime) ) - execution.setIdleTime(ULong.valueOf(stat.operatorStatistics.idleTime)) - execution.setNumWorkers(UInteger.valueOf(stat.operatorStatistics.numWorkers)) - list.add(execution) + runtimeStats.setIdleTime(ULong.valueOf(stat.operatorStatistics.idleTime)) + runtimeStats.setNumWorkers(UInteger.valueOf(stat.operatorStatistics.numWorkers)) + runtimeStatsList.add(runtimeStats) } - WorkflowExecutionsResource.insertWorkflowRuntimeStatistics(list) + + // Insert into operator_runtime_statistics table + WorkflowExecutionsResource.insertOperatorRuntimeStatistics(runtimeStatsList) } catch { case err: Throwable => logger.error("error occurred when storing runtime statistics", err) } diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala index 09015a0a00c..44e51851a2b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/WorkflowExecutionService.scala @@ -8,18 +8,22 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregat import edu.uci.ics.amber.engine.common.Utils import edu.uci.ics.amber.engine.common.client.AmberClient import edu.uci.ics.amber.engine.common.executionruntimestate.ExecutionMetadataStore +import edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorExecutions import edu.uci.ics.texera.web.model.websocket.event.{ TexeraWebSocketEvent, WorkflowErrorEvent, WorkflowStateEvent } import edu.uci.ics.texera.web.model.websocket.request.WorkflowExecuteRequest +import edu.uci.ics.texera.web.resource.dashboard.user.workflow.WorkflowExecutionsResource import edu.uci.ics.texera.web.storage.ExecutionStateStore import edu.uci.ics.texera.web.storage.ExecutionStateStore.updateWorkflowState import edu.uci.ics.texera.web.{ComputingUnitMaster, SubscriptionManager, WebsocketInput} import edu.uci.ics.texera.workflow.{LogicalPlan, WorkflowCompiler} +import org.jooq.types.{UInteger, ULong} import java.net.URI +import java.util import scala.collection.mutable class WorkflowExecutionService( @@ -103,7 +107,10 @@ class WorkflowExecutionService( ) executionReconfigurationService = new ExecutionReconfigurationService(client, executionStateStore, workflow) - executionStatsService = new ExecutionStatsService(client, executionStateStore, workflowContext) + // Create the operatorId to executionId map + val operatorIdToExecutionId = createOperatorIdToExecutionIdMap(workflow) + executionStatsService = + new ExecutionStatsService(client, executionStateStore, operatorIdToExecutionId) executionRuntimeService = new ExecutionRuntimeService( client, executionStateStore, @@ -135,6 +142,27 @@ class WorkflowExecutionService( ) } + private def createOperatorIdToExecutionIdMap(workflow: Workflow): Map[String, ULong] = { + val executionList: util.ArrayList[OperatorExecutions] = new util.ArrayList[OperatorExecutions]() + val operatorIdToExecutionId = scala.collection.mutable.Map[String, ULong]() + + workflow.logicalPlan.operators.foreach { operator => + val operatorId = operator.operatorIdentifier.id + val execution = new OperatorExecutions() + execution.setWorkflowExecutionId(UInteger.valueOf(workflowContext.executionId.id)) + execution.setOperatorId(operatorId) + executionList.add(execution) + } + + val insertedExecutionIds = WorkflowExecutionsResource.insertOperatorExecutions(executionList) + insertedExecutionIds.forEach { + case (operatorId, executionId) => + operatorIdToExecutionId += (operatorId -> executionId) + } + + operatorIdToExecutionId.toMap + } + override def unsubscribeAll(): Unit = { super.unsubscribeAll() if (client != null) { diff --git a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/DefaultCostEstimatorSpec.scala b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/DefaultCostEstimatorSpec.scala index 636a82d7dc8..e3953ebd1bd 100644 --- a/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/DefaultCostEstimatorSpec.scala +++ b/core/amber/src/test/scala/edu/uci/ics/amber/engine/architecture/scheduling/DefaultCostEstimatorSpec.scala @@ -15,6 +15,8 @@ import edu.uci.ics.texera.workflow.LogicalLink import org.jooq.types.{UInteger, ULong} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.jooq.DSLContext +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions.OPERATOR_EXECUTIONS import scala.jdk.CollectionConverters.CollectionHasAsScala @@ -68,34 +70,69 @@ class DefaultCostEstimatorSpec workflowExecution } - private val headerlessCsvOpStatisticsEntry: WorkflowRuntimeStatistics = { - val workflowRuntimeStatistics = new WorkflowRuntimeStatistics - workflowRuntimeStatistics.setOperatorId(headerlessCsvOpDesc.operatorIdentifier.id) - workflowRuntimeStatistics.setWorkflowId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setExecutionId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setDataProcessingTime(ULong.valueOf(100)) - workflowRuntimeStatistics.setControlProcessingTime(ULong.valueOf(100)) - workflowRuntimeStatistics + private val headerlessCsvOperatorExecutionEntry: OperatorExecutions = { + val operatorExecution = new OperatorExecutions + operatorExecution.setWorkflowExecutionId(UInteger.valueOf(1)) + operatorExecution.setOperatorId(headerlessCsvOpDesc.operatorIdentifier.id) + operatorExecution } - private val keywordOpDescStatisticsEntry: WorkflowRuntimeStatistics = { - val workflowRuntimeStatistics = new WorkflowRuntimeStatistics - workflowRuntimeStatistics.setOperatorId(keywordOpDesc.operatorIdentifier.id) - workflowRuntimeStatistics.setWorkflowId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setExecutionId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setDataProcessingTime(ULong.valueOf(300)) - workflowRuntimeStatistics.setControlProcessingTime(ULong.valueOf(300)) - workflowRuntimeStatistics + private val keywordOperatorExecutionEntry: OperatorExecutions = { + val operatorExecution = new OperatorExecutions + operatorExecution.setWorkflowExecutionId(UInteger.valueOf(1)) + operatorExecution.setOperatorId(keywordOpDesc.operatorIdentifier.id) + operatorExecution } - private val groupByOpDescStatisticsEntry: WorkflowRuntimeStatistics = { - val workflowRuntimeStatistics = new WorkflowRuntimeStatistics - workflowRuntimeStatistics.setOperatorId(groupByOpDesc.operatorIdentifier.id) - workflowRuntimeStatistics.setWorkflowId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setExecutionId(UInteger.valueOf(1)) - workflowRuntimeStatistics.setDataProcessingTime(ULong.valueOf(1000)) - workflowRuntimeStatistics.setControlProcessingTime(ULong.valueOf(1000)) - workflowRuntimeStatistics + private val groupByOperatorExecutionEntry: OperatorExecutions = { + val operatorExecution = new OperatorExecutions + operatorExecution.setWorkflowExecutionId(UInteger.valueOf(1)) + operatorExecution.setOperatorId(groupByOpDesc.operatorIdentifier.id) + operatorExecution + } + + private def headerlessCsvOpRuntimeStatisticsEntry( + operatorExecutionId: ULong + ): OperatorRuntimeStatistics = { + val operatorRuntimeStatistics = new OperatorRuntimeStatistics + operatorRuntimeStatistics.setOperatorExecutionId(operatorExecutionId) + operatorRuntimeStatistics.setDataProcessingTime(ULong.valueOf(100)) + operatorRuntimeStatistics.setControlProcessingTime(ULong.valueOf(100)) + operatorRuntimeStatistics + } + + private def keywordOpDescRuntimeStatisticsEntry( + operatorExecutionId: ULong + ): OperatorRuntimeStatistics = { + val operatorRuntimeStatistics = new OperatorRuntimeStatistics + operatorRuntimeStatistics.setOperatorExecutionId(operatorExecutionId) + operatorRuntimeStatistics.setDataProcessingTime(ULong.valueOf(300)) + operatorRuntimeStatistics.setControlProcessingTime(ULong.valueOf(300)) + operatorRuntimeStatistics + } + + private def groupByOpDescRuntimeStatisticsEntry( + operatorExecutionId: ULong + ): OperatorRuntimeStatistics = { + val operatorRuntimeStatistics = new OperatorRuntimeStatistics + operatorRuntimeStatistics.setOperatorExecutionId(operatorExecutionId) + operatorRuntimeStatistics.setDataProcessingTime(ULong.valueOf(1000)) + operatorRuntimeStatistics.setControlProcessingTime(ULong.valueOf(1000)) + operatorRuntimeStatistics + } + + private def insertOperatorExecutionAndGetId( + dslContext: DSLContext, + operatorExecution: OperatorExecutions + ): ULong = { + val record = dslContext + .insertInto(OPERATOR_EXECUTIONS) + .set(OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID, operatorExecution.getWorkflowExecutionId) + .set(OPERATOR_EXECUTIONS.OPERATOR_ID, operatorExecution.getOperatorId) + .returning(OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID) + .fetchOne() + + record.getOperatorExecutionId } override protected def beforeEach(): Unit = { @@ -150,15 +187,30 @@ class DefaultCostEstimatorSpec val workflowDao = new WorkflowDao(getDSLContext.configuration()) val workflowExecutionsDao = new WorkflowExecutionsDao(getDSLContext.configuration()) val workflowVersionDao = new WorkflowVersionDao(getDSLContext.configuration()) - val workflowRuntimeStatisticsDao = - new WorkflowRuntimeStatisticsDao(getDSLContext.configuration()) + val operatorRuntimeStatisticsDao = + new OperatorRuntimeStatisticsDao(getDSLContext.configuration()) userDao.insert(testUser) workflowDao.insert(testWorkflowEntry) workflowVersionDao.insert(testWorkflowVersionEntry) workflowExecutionsDao.insert(testWorkflowExecutionEntry) - workflowRuntimeStatisticsDao.insert(headerlessCsvOpStatisticsEntry) - workflowRuntimeStatisticsDao.insert(keywordOpDescStatisticsEntry) + + val headerlessCsvOperatorExecutionId = insertOperatorExecutionAndGetId( + getDSLContext, + headerlessCsvOperatorExecutionEntry + ) + val keywordOperatorExecutionId = insertOperatorExecutionAndGetId( + getDSLContext, + keywordOperatorExecutionEntry + ) + + val headerlessCsvOpRuntimeStatistics = + headerlessCsvOpRuntimeStatisticsEntry(headerlessCsvOperatorExecutionId) + val keywordOpRuntimeStatistics = + keywordOpDescRuntimeStatisticsEntry(keywordOperatorExecutionId) + + operatorRuntimeStatisticsDao.insert(headerlessCsvOpRuntimeStatistics) + operatorRuntimeStatisticsDao.insert(keywordOpRuntimeStatistics) val costEstimator = new DefaultCostEstimator( workflow.context, @@ -200,16 +252,37 @@ class DefaultCostEstimatorSpec val workflowDao = new WorkflowDao(getDSLContext.configuration()) val workflowExecutionsDao = new WorkflowExecutionsDao(getDSLContext.configuration()) val workflowVersionDao = new WorkflowVersionDao(getDSLContext.configuration()) - val workflowRuntimeStatisticsDao = - new WorkflowRuntimeStatisticsDao(getDSLContext.configuration()) + val operatorRuntimeStatisticsDao = + new OperatorRuntimeStatisticsDao(getDSLContext.configuration()) userDao.insert(testUser) workflowDao.insert(testWorkflowEntry) workflowVersionDao.insert(testWorkflowVersionEntry) workflowExecutionsDao.insert(testWorkflowExecutionEntry) - workflowRuntimeStatisticsDao.insert(headerlessCsvOpStatisticsEntry) - workflowRuntimeStatisticsDao.insert(groupByOpDescStatisticsEntry) - workflowRuntimeStatisticsDao.insert(keywordOpDescStatisticsEntry) + + val headerlessCsvOperatorExecutionId = insertOperatorExecutionAndGetId( + getDSLContext, + headerlessCsvOperatorExecutionEntry + ) + val groupByOperatorExecutionId = insertOperatorExecutionAndGetId( + getDSLContext, + groupByOperatorExecutionEntry + ) + val keywordOperatorExecutionId = insertOperatorExecutionAndGetId( + getDSLContext, + keywordOperatorExecutionEntry + ) + + val headerlessCsvOpRuntimeStatistics = + headerlessCsvOpRuntimeStatisticsEntry(headerlessCsvOperatorExecutionId) + val groupByOpRuntimeStatistics = + groupByOpDescRuntimeStatisticsEntry(groupByOperatorExecutionId) + val keywordOpRuntimeStatistics = + keywordOpDescRuntimeStatisticsEntry(keywordOperatorExecutionId) + + operatorRuntimeStatisticsDao.insert(headerlessCsvOpRuntimeStatistics) + operatorRuntimeStatisticsDao.insert(groupByOpRuntimeStatistics) + operatorRuntimeStatisticsDao.insert(keywordOpRuntimeStatistics) // Should contain two regions, one with CSV->localAgg->globalAgg, another with keyword->sink val searchResult = new CostBasedScheduleGenerator( @@ -230,8 +303,9 @@ class DefaultCostEstimatorSpec val groupByRegionCost = costEstimator.estimate(groupByRegion, 1) - val groupByOperatorCost = (groupByOpDescStatisticsEntry.getControlProcessingTime - .doubleValue() + groupByOpDescStatisticsEntry.getControlProcessingTime.doubleValue()) / 1e9 + val groupByOperatorCost = (groupByOpRuntimeStatistics.getControlProcessingTime + .doubleValue() + groupByOpRuntimeStatistics.getControlProcessingTime + .doubleValue()) / 1e9 // The cost of the first region should be the cost of the GroupBy operator (note the two physical operators for // the GroupBy logical operator have the same cost because we use logical operator in the statistics. @@ -240,8 +314,9 @@ class DefaultCostEstimatorSpec val keywordRegionCost = costEstimator.estimate(keywordRegion, 1) - val keywordOperatorCost = (keywordOpDescStatisticsEntry.getControlProcessingTime - .doubleValue() + keywordOpDescStatisticsEntry.getControlProcessingTime.doubleValue()) / 1e9 + val keywordOperatorCost = (keywordOpRuntimeStatistics.getControlProcessingTime + .doubleValue() + keywordOpRuntimeStatistics.getControlProcessingTime + .doubleValue()) / 1e9 // The cost of the second region should be the cost of the keyword operator, since the sink operator has the same // logical operator as the keyword operator. diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Indexes.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Indexes.java index fc916ac9d54..60f5acc26f2 100644 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Indexes.java +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Indexes.java @@ -7,6 +7,8 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.Dataset; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetVersion; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.Project; import edu.uci.ics.texera.dao.jooq.generated.tables.ProjectUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.PublicProject; @@ -16,7 +18,6 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowExecutions; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfProject; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfUser; -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserClones; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserLikes; @@ -46,6 +47,9 @@ public class Indexes { public static final Index DATASET_VERSION_DID = Indexes0.DATASET_VERSION_DID; public static final Index DATASET_VERSION_IDX_DATASET_VERSION_NAME = Indexes0.DATASET_VERSION_IDX_DATASET_VERSION_NAME; public static final Index DATASET_VERSION_PRIMARY = Indexes0.DATASET_VERSION_PRIMARY; + public static final Index OPERATOR_EXECUTIONS_PRIMARY = Indexes0.OPERATOR_EXECUTIONS_PRIMARY; + public static final Index OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID = Indexes0.OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID; + public static final Index OPERATOR_RUNTIME_STATISTICS_PRIMARY = Indexes0.OPERATOR_RUNTIME_STATISTICS_PRIMARY; public static final Index PROJECT_IDX_USER_PROJECT_NAME_DESCRIPTION = Indexes0.PROJECT_IDX_USER_PROJECT_NAME_DESCRIPTION; public static final Index PROJECT_OWNER_ID = Indexes0.PROJECT_OWNER_ID; public static final Index PROJECT_PRIMARY = Indexes0.PROJECT_PRIMARY; @@ -66,8 +70,6 @@ public class Indexes { public static final Index WORKFLOW_OF_PROJECT_PRIMARY = Indexes0.WORKFLOW_OF_PROJECT_PRIMARY; public static final Index WORKFLOW_OF_USER_PRIMARY = Indexes0.WORKFLOW_OF_USER_PRIMARY; public static final Index WORKFLOW_OF_USER_WID = Indexes0.WORKFLOW_OF_USER_WID; - public static final Index WORKFLOW_RUNTIME_STATISTICS_EXECUTION_ID = Indexes0.WORKFLOW_RUNTIME_STATISTICS_EXECUTION_ID; - public static final Index WORKFLOW_RUNTIME_STATISTICS_PRIMARY = Indexes0.WORKFLOW_RUNTIME_STATISTICS_PRIMARY; public static final Index WORKFLOW_USER_ACCESS_PRIMARY = Indexes0.WORKFLOW_USER_ACCESS_PRIMARY; public static final Index WORKFLOW_USER_ACCESS_WID = Indexes0.WORKFLOW_USER_ACCESS_WID; public static final Index WORKFLOW_USER_CLONES_PRIMARY = Indexes0.WORKFLOW_USER_CLONES_PRIMARY; @@ -91,6 +93,9 @@ private static class Indexes0 { public static Index DATASET_VERSION_DID = Internal.createIndex("did", DatasetVersion.DATASET_VERSION, new OrderField[] { DatasetVersion.DATASET_VERSION.DID }, false); public static Index DATASET_VERSION_IDX_DATASET_VERSION_NAME = Internal.createIndex("idx_dataset_version_name", DatasetVersion.DATASET_VERSION, new OrderField[] { DatasetVersion.DATASET_VERSION.NAME }, false); public static Index DATASET_VERSION_PRIMARY = Internal.createIndex("PRIMARY", DatasetVersion.DATASET_VERSION, new OrderField[] { DatasetVersion.DATASET_VERSION.DVID }, true); + public static Index OPERATOR_EXECUTIONS_PRIMARY = Internal.createIndex("PRIMARY", OperatorExecutions.OPERATOR_EXECUTIONS, new OrderField[] { OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID }, true); + public static Index OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID = Internal.createIndex("workflow_execution_id", OperatorExecutions.OPERATOR_EXECUTIONS, new OrderField[] { OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID, OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_ID }, true); + public static Index OPERATOR_RUNTIME_STATISTICS_PRIMARY = Internal.createIndex("PRIMARY", OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, new OrderField[] { OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID, OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.TIME }, true); public static Index PROJECT_IDX_USER_PROJECT_NAME_DESCRIPTION = Internal.createIndex("idx_user_project_name_description", Project.PROJECT, new OrderField[] { Project.PROJECT.NAME, Project.PROJECT.DESCRIPTION }, false); public static Index PROJECT_OWNER_ID = Internal.createIndex("owner_id", Project.PROJECT, new OrderField[] { Project.PROJECT.OWNER_ID, Project.PROJECT.NAME }, true); public static Index PROJECT_PRIMARY = Internal.createIndex("PRIMARY", Project.PROJECT, new OrderField[] { Project.PROJECT.PID }, true); @@ -111,8 +116,6 @@ private static class Indexes0 { public static Index WORKFLOW_OF_PROJECT_PRIMARY = Internal.createIndex("PRIMARY", WorkflowOfProject.WORKFLOW_OF_PROJECT, new OrderField[] { WorkflowOfProject.WORKFLOW_OF_PROJECT.WID, WorkflowOfProject.WORKFLOW_OF_PROJECT.PID }, true); public static Index WORKFLOW_OF_USER_PRIMARY = Internal.createIndex("PRIMARY", WorkflowOfUser.WORKFLOW_OF_USER, new OrderField[] { WorkflowOfUser.WORKFLOW_OF_USER.UID, WorkflowOfUser.WORKFLOW_OF_USER.WID }, true); public static Index WORKFLOW_OF_USER_WID = Internal.createIndex("wid", WorkflowOfUser.WORKFLOW_OF_USER, new OrderField[] { WorkflowOfUser.WORKFLOW_OF_USER.WID }, false); - public static Index WORKFLOW_RUNTIME_STATISTICS_EXECUTION_ID = Internal.createIndex("execution_id", WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, new OrderField[] { WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID }, false); - public static Index WORKFLOW_RUNTIME_STATISTICS_PRIMARY = Internal.createIndex("PRIMARY", WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, new OrderField[] { WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.TIME }, true); public static Index WORKFLOW_USER_ACCESS_PRIMARY = Internal.createIndex("PRIMARY", WorkflowUserAccess.WORKFLOW_USER_ACCESS, new OrderField[] { WorkflowUserAccess.WORKFLOW_USER_ACCESS.UID, WorkflowUserAccess.WORKFLOW_USER_ACCESS.WID }, true); public static Index WORKFLOW_USER_ACCESS_WID = Internal.createIndex("wid", WorkflowUserAccess.WORKFLOW_USER_ACCESS, new OrderField[] { WorkflowUserAccess.WORKFLOW_USER_ACCESS.WID }, false); public static Index WORKFLOW_USER_CLONES_PRIMARY = Internal.createIndex("PRIMARY", WorkflowUserClones.WORKFLOW_USER_CLONES, new OrderField[] { WorkflowUserClones.WORKFLOW_USER_CLONES.UID, WorkflowUserClones.WORKFLOW_USER_CLONES.WID }, true); diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Keys.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Keys.java index 12d73c1a08c..3064b7f28b8 100644 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Keys.java +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Keys.java @@ -7,6 +7,8 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.Dataset; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetVersion; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.Project; import edu.uci.ics.texera.dao.jooq.generated.tables.ProjectUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.PublicProject; @@ -16,7 +18,6 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowExecutions; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfProject; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfUser; -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserClones; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserLikes; @@ -25,6 +26,8 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.records.DatasetRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.DatasetUserAccessRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.DatasetVersionRecord; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorExecutionsRecord; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorRuntimeStatisticsRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.ProjectRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.ProjectUserAccessRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.PublicProjectRecord; @@ -34,7 +37,6 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowOfProjectRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowOfUserRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowRecord; -import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowRuntimeStatisticsRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowUserAccessRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowUserClonesRecord; import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowUserLikesRecord; @@ -46,6 +48,7 @@ import org.jooq.UniqueKey; import org.jooq.impl.Internal; import org.jooq.types.UInteger; +import org.jooq.types.ULong; /** @@ -61,6 +64,7 @@ public class Keys { public static final Identity IDENTITY_DATASET = Identities0.IDENTITY_DATASET; public static final Identity IDENTITY_DATASET_VERSION = Identities0.IDENTITY_DATASET_VERSION; + public static final Identity IDENTITY_OPERATOR_EXECUTIONS = Identities0.IDENTITY_OPERATOR_EXECUTIONS; public static final Identity IDENTITY_PROJECT = Identities0.IDENTITY_PROJECT; public static final Identity IDENTITY_USER = Identities0.IDENTITY_USER; public static final Identity IDENTITY_WORKFLOW = Identities0.IDENTITY_WORKFLOW; @@ -74,6 +78,9 @@ public class Keys { public static final UniqueKey KEY_DATASET_PRIMARY = UniqueKeys0.KEY_DATASET_PRIMARY; public static final UniqueKey KEY_DATASET_USER_ACCESS_PRIMARY = UniqueKeys0.KEY_DATASET_USER_ACCESS_PRIMARY; public static final UniqueKey KEY_DATASET_VERSION_PRIMARY = UniqueKeys0.KEY_DATASET_VERSION_PRIMARY; + public static final UniqueKey KEY_OPERATOR_EXECUTIONS_PRIMARY = UniqueKeys0.KEY_OPERATOR_EXECUTIONS_PRIMARY; + public static final UniqueKey KEY_OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID = UniqueKeys0.KEY_OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID; + public static final UniqueKey KEY_OPERATOR_RUNTIME_STATISTICS_PRIMARY = UniqueKeys0.KEY_OPERATOR_RUNTIME_STATISTICS_PRIMARY; public static final UniqueKey KEY_PROJECT_PRIMARY = UniqueKeys0.KEY_PROJECT_PRIMARY; public static final UniqueKey KEY_PROJECT_OWNER_ID = UniqueKeys0.KEY_PROJECT_OWNER_ID; public static final UniqueKey KEY_PROJECT_USER_ACCESS_PRIMARY = UniqueKeys0.KEY_PROJECT_USER_ACCESS_PRIMARY; @@ -86,7 +93,6 @@ public class Keys { public static final UniqueKey KEY_WORKFLOW_EXECUTIONS_PRIMARY = UniqueKeys0.KEY_WORKFLOW_EXECUTIONS_PRIMARY; public static final UniqueKey KEY_WORKFLOW_OF_PROJECT_PRIMARY = UniqueKeys0.KEY_WORKFLOW_OF_PROJECT_PRIMARY; public static final UniqueKey KEY_WORKFLOW_OF_USER_PRIMARY = UniqueKeys0.KEY_WORKFLOW_OF_USER_PRIMARY; - public static final UniqueKey KEY_WORKFLOW_RUNTIME_STATISTICS_PRIMARY = UniqueKeys0.KEY_WORKFLOW_RUNTIME_STATISTICS_PRIMARY; public static final UniqueKey KEY_WORKFLOW_USER_ACCESS_PRIMARY = UniqueKeys0.KEY_WORKFLOW_USER_ACCESS_PRIMARY; public static final UniqueKey KEY_WORKFLOW_USER_CLONES_PRIMARY = UniqueKeys0.KEY_WORKFLOW_USER_CLONES_PRIMARY; public static final UniqueKey KEY_WORKFLOW_USER_LIKES_PRIMARY = UniqueKeys0.KEY_WORKFLOW_USER_LIKES_PRIMARY; @@ -101,6 +107,8 @@ public class Keys { public static final ForeignKey DATASET_USER_ACCESS_IBFK_1 = ForeignKeys0.DATASET_USER_ACCESS_IBFK_1; public static final ForeignKey DATASET_USER_ACCESS_IBFK_2 = ForeignKeys0.DATASET_USER_ACCESS_IBFK_2; public static final ForeignKey DATASET_VERSION_IBFK_1 = ForeignKeys0.DATASET_VERSION_IBFK_1; + public static final ForeignKey OPERATOR_EXECUTIONS_IBFK_1 = ForeignKeys0.OPERATOR_EXECUTIONS_IBFK_1; + public static final ForeignKey OPERATOR_RUNTIME_STATISTICS_IBFK_1 = ForeignKeys0.OPERATOR_RUNTIME_STATISTICS_IBFK_1; public static final ForeignKey PROJECT_IBFK_1 = ForeignKeys0.PROJECT_IBFK_1; public static final ForeignKey PROJECT_USER_ACCESS_IBFK_1 = ForeignKeys0.PROJECT_USER_ACCESS_IBFK_1; public static final ForeignKey PROJECT_USER_ACCESS_IBFK_2 = ForeignKeys0.PROJECT_USER_ACCESS_IBFK_2; @@ -112,8 +120,6 @@ public class Keys { public static final ForeignKey WORKFLOW_OF_PROJECT_IBFK_2 = ForeignKeys0.WORKFLOW_OF_PROJECT_IBFK_2; public static final ForeignKey WORKFLOW_OF_USER_IBFK_1 = ForeignKeys0.WORKFLOW_OF_USER_IBFK_1; public static final ForeignKey WORKFLOW_OF_USER_IBFK_2 = ForeignKeys0.WORKFLOW_OF_USER_IBFK_2; - public static final ForeignKey WORKFLOW_RUNTIME_STATISTICS_IBFK_1 = ForeignKeys0.WORKFLOW_RUNTIME_STATISTICS_IBFK_1; - public static final ForeignKey WORKFLOW_RUNTIME_STATISTICS_IBFK_2 = ForeignKeys0.WORKFLOW_RUNTIME_STATISTICS_IBFK_2; public static final ForeignKey WORKFLOW_USER_ACCESS_IBFK_1 = ForeignKeys0.WORKFLOW_USER_ACCESS_IBFK_1; public static final ForeignKey WORKFLOW_USER_ACCESS_IBFK_2 = ForeignKeys0.WORKFLOW_USER_ACCESS_IBFK_2; public static final ForeignKey WORKFLOW_USER_CLONES_IBFK_1 = ForeignKeys0.WORKFLOW_USER_CLONES_IBFK_1; @@ -130,6 +136,7 @@ public class Keys { private static class Identities0 { public static Identity IDENTITY_DATASET = Internal.createIdentity(Dataset.DATASET, Dataset.DATASET.DID); public static Identity IDENTITY_DATASET_VERSION = Internal.createIdentity(DatasetVersion.DATASET_VERSION, DatasetVersion.DATASET_VERSION.DVID); + public static Identity IDENTITY_OPERATOR_EXECUTIONS = Internal.createIdentity(OperatorExecutions.OPERATOR_EXECUTIONS, OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID); public static Identity IDENTITY_PROJECT = Internal.createIdentity(Project.PROJECT, Project.PROJECT.PID); public static Identity IDENTITY_USER = Internal.createIdentity(User.USER, User.USER.UID); public static Identity IDENTITY_WORKFLOW = Internal.createIdentity(Workflow.WORKFLOW, Workflow.WORKFLOW.WID); @@ -141,6 +148,9 @@ private static class UniqueKeys0 { public static final UniqueKey KEY_DATASET_PRIMARY = Internal.createUniqueKey(Dataset.DATASET, "KEY_dataset_PRIMARY", Dataset.DATASET.DID); public static final UniqueKey KEY_DATASET_USER_ACCESS_PRIMARY = Internal.createUniqueKey(DatasetUserAccess.DATASET_USER_ACCESS, "KEY_dataset_user_access_PRIMARY", DatasetUserAccess.DATASET_USER_ACCESS.DID, DatasetUserAccess.DATASET_USER_ACCESS.UID); public static final UniqueKey KEY_DATASET_VERSION_PRIMARY = Internal.createUniqueKey(DatasetVersion.DATASET_VERSION, "KEY_dataset_version_PRIMARY", DatasetVersion.DATASET_VERSION.DVID); + public static final UniqueKey KEY_OPERATOR_EXECUTIONS_PRIMARY = Internal.createUniqueKey(OperatorExecutions.OPERATOR_EXECUTIONS, "KEY_operator_executions_PRIMARY", OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID); + public static final UniqueKey KEY_OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID = Internal.createUniqueKey(OperatorExecutions.OPERATOR_EXECUTIONS, "KEY_operator_executions_workflow_execution_id", OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID, OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_ID); + public static final UniqueKey KEY_OPERATOR_RUNTIME_STATISTICS_PRIMARY = Internal.createUniqueKey(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, "KEY_operator_runtime_statistics_PRIMARY", OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID, OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.TIME); public static final UniqueKey KEY_PROJECT_PRIMARY = Internal.createUniqueKey(Project.PROJECT, "KEY_project_PRIMARY", Project.PROJECT.PID); public static final UniqueKey KEY_PROJECT_OWNER_ID = Internal.createUniqueKey(Project.PROJECT, "KEY_project_owner_id", Project.PROJECT.OWNER_ID, Project.PROJECT.NAME); public static final UniqueKey KEY_PROJECT_USER_ACCESS_PRIMARY = Internal.createUniqueKey(ProjectUserAccess.PROJECT_USER_ACCESS, "KEY_project_user_access_PRIMARY", ProjectUserAccess.PROJECT_USER_ACCESS.UID, ProjectUserAccess.PROJECT_USER_ACCESS.PID); @@ -153,7 +163,6 @@ private static class UniqueKeys0 { public static final UniqueKey KEY_WORKFLOW_EXECUTIONS_PRIMARY = Internal.createUniqueKey(WorkflowExecutions.WORKFLOW_EXECUTIONS, "KEY_workflow_executions_PRIMARY", WorkflowExecutions.WORKFLOW_EXECUTIONS.EID); public static final UniqueKey KEY_WORKFLOW_OF_PROJECT_PRIMARY = Internal.createUniqueKey(WorkflowOfProject.WORKFLOW_OF_PROJECT, "KEY_workflow_of_project_PRIMARY", WorkflowOfProject.WORKFLOW_OF_PROJECT.WID, WorkflowOfProject.WORKFLOW_OF_PROJECT.PID); public static final UniqueKey KEY_WORKFLOW_OF_USER_PRIMARY = Internal.createUniqueKey(WorkflowOfUser.WORKFLOW_OF_USER, "KEY_workflow_of_user_PRIMARY", WorkflowOfUser.WORKFLOW_OF_USER.UID, WorkflowOfUser.WORKFLOW_OF_USER.WID); - public static final UniqueKey KEY_WORKFLOW_RUNTIME_STATISTICS_PRIMARY = Internal.createUniqueKey(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, "KEY_workflow_runtime_statistics_PRIMARY", WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.TIME); public static final UniqueKey KEY_WORKFLOW_USER_ACCESS_PRIMARY = Internal.createUniqueKey(WorkflowUserAccess.WORKFLOW_USER_ACCESS, "KEY_workflow_user_access_PRIMARY", WorkflowUserAccess.WORKFLOW_USER_ACCESS.UID, WorkflowUserAccess.WORKFLOW_USER_ACCESS.WID); public static final UniqueKey KEY_WORKFLOW_USER_CLONES_PRIMARY = Internal.createUniqueKey(WorkflowUserClones.WORKFLOW_USER_CLONES, "KEY_workflow_user_clones_PRIMARY", WorkflowUserClones.WORKFLOW_USER_CLONES.UID, WorkflowUserClones.WORKFLOW_USER_CLONES.WID); public static final UniqueKey KEY_WORKFLOW_USER_LIKES_PRIMARY = Internal.createUniqueKey(WorkflowUserLikes.WORKFLOW_USER_LIKES, "KEY_workflow_user_likes_PRIMARY", WorkflowUserLikes.WORKFLOW_USER_LIKES.UID, WorkflowUserLikes.WORKFLOW_USER_LIKES.WID); @@ -166,6 +175,8 @@ private static class ForeignKeys0 { public static final ForeignKey DATASET_USER_ACCESS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_DATASET_PRIMARY, DatasetUserAccess.DATASET_USER_ACCESS, "dataset_user_access_ibfk_1", DatasetUserAccess.DATASET_USER_ACCESS.DID); public static final ForeignKey DATASET_USER_ACCESS_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, DatasetUserAccess.DATASET_USER_ACCESS, "dataset_user_access_ibfk_2", DatasetUserAccess.DATASET_USER_ACCESS.UID); public static final ForeignKey DATASET_VERSION_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_DATASET_PRIMARY, DatasetVersion.DATASET_VERSION, "dataset_version_ibfk_1", DatasetVersion.DATASET_VERSION.DID); + public static final ForeignKey OPERATOR_EXECUTIONS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_WORKFLOW_EXECUTIONS_PRIMARY, OperatorExecutions.OPERATOR_EXECUTIONS, "operator_executions_ibfk_1", OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID); + public static final ForeignKey OPERATOR_RUNTIME_STATISTICS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_OPERATOR_EXECUTIONS_PRIMARY, OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, "operator_runtime_statistics_ibfk_1", OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID); public static final ForeignKey PROJECT_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, Project.PROJECT, "project_ibfk_1", Project.PROJECT.OWNER_ID); public static final ForeignKey PROJECT_USER_ACCESS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, ProjectUserAccess.PROJECT_USER_ACCESS, "project_user_access_ibfk_1", ProjectUserAccess.PROJECT_USER_ACCESS.UID); public static final ForeignKey PROJECT_USER_ACCESS_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_PROJECT_PRIMARY, ProjectUserAccess.PROJECT_USER_ACCESS, "project_user_access_ibfk_2", ProjectUserAccess.PROJECT_USER_ACCESS.PID); @@ -177,8 +188,6 @@ private static class ForeignKeys0 { public static final ForeignKey WORKFLOW_OF_PROJECT_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_PROJECT_PRIMARY, WorkflowOfProject.WORKFLOW_OF_PROJECT, "workflow_of_project_ibfk_2", WorkflowOfProject.WORKFLOW_OF_PROJECT.PID); public static final ForeignKey WORKFLOW_OF_USER_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, WorkflowOfUser.WORKFLOW_OF_USER, "workflow_of_user_ibfk_1", WorkflowOfUser.WORKFLOW_OF_USER.UID); public static final ForeignKey WORKFLOW_OF_USER_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_WORKFLOW_PRIMARY, WorkflowOfUser.WORKFLOW_OF_USER, "workflow_of_user_ibfk_2", WorkflowOfUser.WORKFLOW_OF_USER.WID); - public static final ForeignKey WORKFLOW_RUNTIME_STATISTICS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_WORKFLOW_PRIMARY, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, "workflow_runtime_statistics_ibfk_1", WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID); - public static final ForeignKey WORKFLOW_RUNTIME_STATISTICS_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_WORKFLOW_EXECUTIONS_PRIMARY, WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, "workflow_runtime_statistics_ibfk_2", WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID); public static final ForeignKey WORKFLOW_USER_ACCESS_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, WorkflowUserAccess.WORKFLOW_USER_ACCESS, "workflow_user_access_ibfk_1", WorkflowUserAccess.WORKFLOW_USER_ACCESS.UID); public static final ForeignKey WORKFLOW_USER_ACCESS_IBFK_2 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_WORKFLOW_PRIMARY, WorkflowUserAccess.WORKFLOW_USER_ACCESS, "workflow_user_access_ibfk_2", WorkflowUserAccess.WORKFLOW_USER_ACCESS.WID); public static final ForeignKey WORKFLOW_USER_CLONES_IBFK_1 = Internal.createForeignKey(edu.uci.ics.texera.dao.jooq.generated.Keys.KEY_USER_PRIMARY, WorkflowUserClones.WORKFLOW_USER_CLONES, "workflow_user_clones_ibfk_1", WorkflowUserClones.WORKFLOW_USER_CLONES.UID); diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Tables.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Tables.java index 33f52316ad9..5cb9375ea0f 100644 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Tables.java +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/Tables.java @@ -7,6 +7,8 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.Dataset; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetVersion; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.Project; import edu.uci.ics.texera.dao.jooq.generated.tables.ProjectUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.PublicProject; @@ -16,7 +18,6 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowExecutions; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfProject; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfUser; -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserActivity; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserClones; @@ -46,6 +47,16 @@ public class Tables { */ public static final DatasetVersion DATASET_VERSION = DatasetVersion.DATASET_VERSION; + /** + * The table texera_db.operator_executions. + */ + public static final OperatorExecutions OPERATOR_EXECUTIONS = OperatorExecutions.OPERATOR_EXECUTIONS; + + /** + * The table texera_db.operator_runtime_statistics. + */ + public static final OperatorRuntimeStatistics OPERATOR_RUNTIME_STATISTICS = OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS; + /** * The table texera_db.project. */ @@ -91,11 +102,6 @@ public class Tables { */ public static final WorkflowOfUser WORKFLOW_OF_USER = WorkflowOfUser.WORKFLOW_OF_USER; - /** - * The table texera_db.workflow_runtime_statistics. - */ - public static final WorkflowRuntimeStatistics WORKFLOW_RUNTIME_STATISTICS = WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS; - /** * The table texera_db.workflow_user_access. */ diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/TexeraDb.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/TexeraDb.java index 84e9f00a718..f138504cef0 100644 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/TexeraDb.java +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/TexeraDb.java @@ -7,6 +7,8 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.Dataset; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.DatasetVersion; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.Project; import edu.uci.ics.texera.dao.jooq.generated.tables.ProjectUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.PublicProject; @@ -16,7 +18,6 @@ import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowExecutions; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfProject; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfUser; -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserAccess; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserActivity; import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowUserClones; @@ -39,7 +40,7 @@ @SuppressWarnings({ "all", "unchecked", "rawtypes" }) public class TexeraDb extends SchemaImpl { - private static final long serialVersionUID = 1224383013; + private static final long serialVersionUID = -2031114180; /** * The reference instance of texera_db @@ -61,6 +62,16 @@ public class TexeraDb extends SchemaImpl { */ public final DatasetVersion DATASET_VERSION = edu.uci.ics.texera.dao.jooq.generated.tables.DatasetVersion.DATASET_VERSION; + /** + * The table texera_db.operator_executions. + */ + public final OperatorExecutions OPERATOR_EXECUTIONS = edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions.OPERATOR_EXECUTIONS; + + /** + * The table texera_db.operator_runtime_statistics. + */ + public final OperatorRuntimeStatistics OPERATOR_RUNTIME_STATISTICS = edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS; + /** * The table texera_db.project. */ @@ -106,11 +117,6 @@ public class TexeraDb extends SchemaImpl { */ public final WorkflowOfUser WORKFLOW_OF_USER = edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowOfUser.WORKFLOW_OF_USER; - /** - * The table texera_db.workflow_runtime_statistics. - */ - public final WorkflowRuntimeStatistics WORKFLOW_RUNTIME_STATISTICS = edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS; - /** * The table texera_db.workflow_user_access. */ @@ -166,6 +172,8 @@ private final List> getTables0() { Dataset.DATASET, DatasetUserAccess.DATASET_USER_ACCESS, DatasetVersion.DATASET_VERSION, + OperatorExecutions.OPERATOR_EXECUTIONS, + OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, Project.PROJECT, ProjectUserAccess.PROJECT_USER_ACCESS, PublicProject.PUBLIC_PROJECT, @@ -175,7 +183,6 @@ private final List> getTables0() { WorkflowExecutions.WORKFLOW_EXECUTIONS, WorkflowOfProject.WORKFLOW_OF_PROJECT, WorkflowOfUser.WORKFLOW_OF_USER, - WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, WorkflowUserAccess.WORKFLOW_USER_ACCESS, WorkflowUserActivity.WORKFLOW_USER_ACTIVITY, WorkflowUserClones.WORKFLOW_USER_CLONES, diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorExecutions.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorExecutions.java new file mode 100644 index 00000000000..68c4a86caf8 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorExecutions.java @@ -0,0 +1,169 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables; + + +import edu.uci.ics.texera.dao.jooq.generated.Indexes; +import edu.uci.ics.texera.dao.jooq.generated.Keys; +import edu.uci.ics.texera.dao.jooq.generated.TexeraDb; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorExecutionsRecord; + +import java.util.Arrays; +import java.util.List; + +import org.jooq.Field; +import org.jooq.ForeignKey; +import org.jooq.Identity; +import org.jooq.Index; +import org.jooq.Name; +import org.jooq.Record; +import org.jooq.Row3; +import org.jooq.Schema; +import org.jooq.Table; +import org.jooq.TableField; +import org.jooq.UniqueKey; +import org.jooq.impl.DSL; +import org.jooq.impl.TableImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorExecutions extends TableImpl { + + private static final long serialVersionUID = 1254946167; + + /** + * The reference instance of texera_db.operator_executions + */ + public static final OperatorExecutions OPERATOR_EXECUTIONS = new OperatorExecutions(); + + /** + * The class holding records for this type + */ + @Override + public Class getRecordType() { + return OperatorExecutionsRecord.class; + } + + /** + * The column texera_db.operator_executions.operator_execution_id. + */ + public final TableField OPERATOR_EXECUTION_ID = createField(DSL.name("operator_execution_id"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).identity(true), this, ""); + + /** + * The column texera_db.operator_executions.workflow_execution_id. + */ + public final TableField WORKFLOW_EXECUTION_ID = createField(DSL.name("workflow_execution_id"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false), this, ""); + + /** + * The column texera_db.operator_executions.operator_id. + */ + public final TableField OPERATOR_ID = createField(DSL.name("operator_id"), org.jooq.impl.SQLDataType.VARCHAR(100).nullable(false), this, ""); + + /** + * Create a texera_db.operator_executions table reference + */ + public OperatorExecutions() { + this(DSL.name("operator_executions"), null); + } + + /** + * Create an aliased texera_db.operator_executions table reference + */ + public OperatorExecutions(String alias) { + this(DSL.name(alias), OPERATOR_EXECUTIONS); + } + + /** + * Create an aliased texera_db.operator_executions table reference + */ + public OperatorExecutions(Name alias) { + this(alias, OPERATOR_EXECUTIONS); + } + + private OperatorExecutions(Name alias, Table aliased) { + this(alias, aliased, null); + } + + private OperatorExecutions(Name alias, Table aliased, Field[] parameters) { + super(alias, null, aliased, parameters, DSL.comment("")); + } + + public OperatorExecutions(Table child, ForeignKey key) { + super(child, key, OPERATOR_EXECUTIONS); + } + + @Override + public Schema getSchema() { + return TexeraDb.TEXERA_DB; + } + + @Override + public List getIndexes() { + return Arrays.asList(Indexes.OPERATOR_EXECUTIONS_PRIMARY, Indexes.OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID); + } + + @Override + public Identity getIdentity() { + return Keys.IDENTITY_OPERATOR_EXECUTIONS; + } + + @Override + public UniqueKey getPrimaryKey() { + return Keys.KEY_OPERATOR_EXECUTIONS_PRIMARY; + } + + @Override + public List> getKeys() { + return Arrays.>asList(Keys.KEY_OPERATOR_EXECUTIONS_PRIMARY, Keys.KEY_OPERATOR_EXECUTIONS_WORKFLOW_EXECUTION_ID); + } + + @Override + public List> getReferences() { + return Arrays.>asList(Keys.OPERATOR_EXECUTIONS_IBFK_1); + } + + public WorkflowExecutions workflowExecutions() { + return new WorkflowExecutions(this, Keys.OPERATOR_EXECUTIONS_IBFK_1); + } + + @Override + public OperatorExecutions as(String alias) { + return new OperatorExecutions(DSL.name(alias), this); + } + + @Override + public OperatorExecutions as(Name alias) { + return new OperatorExecutions(alias, this); + } + + /** + * Rename this table + */ + @Override + public OperatorExecutions rename(String name) { + return new OperatorExecutions(DSL.name(name), null); + } + + /** + * Rename this table + */ + @Override + public OperatorExecutions rename(Name name) { + return new OperatorExecutions(name, null); + } + + // ------------------------------------------------------------------------- + // Row3 type methods + // ------------------------------------------------------------------------- + + @Override + public Row3 fieldsRow() { + return (Row3) super.fieldsRow(); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorRuntimeStatistics.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorRuntimeStatistics.java new file mode 100644 index 00000000000..940d6a515a9 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/OperatorRuntimeStatistics.java @@ -0,0 +1,194 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables; + + +import edu.uci.ics.texera.dao.jooq.generated.Indexes; +import edu.uci.ics.texera.dao.jooq.generated.Keys; +import edu.uci.ics.texera.dao.jooq.generated.TexeraDb; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorRuntimeStatisticsRecord; + +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; + +import org.jooq.Field; +import org.jooq.ForeignKey; +import org.jooq.Index; +import org.jooq.Name; +import org.jooq.Record; +import org.jooq.Row9; +import org.jooq.Schema; +import org.jooq.Table; +import org.jooq.TableField; +import org.jooq.UniqueKey; +import org.jooq.impl.DSL; +import org.jooq.impl.TableImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorRuntimeStatistics extends TableImpl { + + private static final long serialVersionUID = -697570718; + + /** + * The reference instance of texera_db.operator_runtime_statistics + */ + public static final OperatorRuntimeStatistics OPERATOR_RUNTIME_STATISTICS = new OperatorRuntimeStatistics(); + + /** + * The class holding records for this type + */ + @Override + public Class getRecordType() { + return OperatorRuntimeStatisticsRecord.class; + } + + /** + * The column texera_db.operator_runtime_statistics.operator_execution_id. + */ + public final TableField OPERATOR_EXECUTION_ID = createField(DSL.name("operator_execution_id"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.time. + */ + public final TableField TIME = createField(DSL.name("time"), org.jooq.impl.SQLDataType.TIMESTAMP.nullable(false).defaultValue(org.jooq.impl.DSL.field("CURRENT_TIMESTAMP(6)", org.jooq.impl.SQLDataType.TIMESTAMP)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.input_tuple_cnt. + */ + public final TableField INPUT_TUPLE_CNT = createField(DSL.name("input_tuple_cnt"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.output_tuple_cnt. + */ + public final TableField OUTPUT_TUPLE_CNT = createField(DSL.name("output_tuple_cnt"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.status. + */ + public final TableField STATUS = createField(DSL.name("status"), org.jooq.impl.SQLDataType.TINYINT.nullable(false).defaultValue(org.jooq.impl.DSL.inline("1", org.jooq.impl.SQLDataType.TINYINT)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.data_processing_time. + */ + public final TableField DATA_PROCESSING_TIME = createField(DSL.name("data_processing_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.control_processing_time. + */ + public final TableField CONTROL_PROCESSING_TIME = createField(DSL.name("control_processing_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.idle_time. + */ + public final TableField IDLE_TIME = createField(DSL.name("idle_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); + + /** + * The column texera_db.operator_runtime_statistics.num_workers. + */ + public final TableField NUM_WORKERS = createField(DSL.name("num_workers"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.INTEGERUNSIGNED)), this, ""); + + /** + * Create a texera_db.operator_runtime_statistics table reference + */ + public OperatorRuntimeStatistics() { + this(DSL.name("operator_runtime_statistics"), null); + } + + /** + * Create an aliased texera_db.operator_runtime_statistics table reference + */ + public OperatorRuntimeStatistics(String alias) { + this(DSL.name(alias), OPERATOR_RUNTIME_STATISTICS); + } + + /** + * Create an aliased texera_db.operator_runtime_statistics table reference + */ + public OperatorRuntimeStatistics(Name alias) { + this(alias, OPERATOR_RUNTIME_STATISTICS); + } + + private OperatorRuntimeStatistics(Name alias, Table aliased) { + this(alias, aliased, null); + } + + private OperatorRuntimeStatistics(Name alias, Table aliased, Field[] parameters) { + super(alias, null, aliased, parameters, DSL.comment("")); + } + + public OperatorRuntimeStatistics(Table child, ForeignKey key) { + super(child, key, OPERATOR_RUNTIME_STATISTICS); + } + + @Override + public Schema getSchema() { + return TexeraDb.TEXERA_DB; + } + + @Override + public List getIndexes() { + return Arrays.asList(Indexes.OPERATOR_RUNTIME_STATISTICS_PRIMARY); + } + + @Override + public UniqueKey getPrimaryKey() { + return Keys.KEY_OPERATOR_RUNTIME_STATISTICS_PRIMARY; + } + + @Override + public List> getKeys() { + return Arrays.>asList(Keys.KEY_OPERATOR_RUNTIME_STATISTICS_PRIMARY); + } + + @Override + public List> getReferences() { + return Arrays.>asList(Keys.OPERATOR_RUNTIME_STATISTICS_IBFK_1); + } + + public OperatorExecutions operatorExecutions() { + return new OperatorExecutions(this, Keys.OPERATOR_RUNTIME_STATISTICS_IBFK_1); + } + + @Override + public OperatorRuntimeStatistics as(String alias) { + return new OperatorRuntimeStatistics(DSL.name(alias), this); + } + + @Override + public OperatorRuntimeStatistics as(Name alias) { + return new OperatorRuntimeStatistics(alias, this); + } + + /** + * Rename this table + */ + @Override + public OperatorRuntimeStatistics rename(String name) { + return new OperatorRuntimeStatistics(DSL.name(name), null); + } + + /** + * Rename this table + */ + @Override + public OperatorRuntimeStatistics rename(Name name) { + return new OperatorRuntimeStatistics(name, null); + } + + // ------------------------------------------------------------------------- + // Row9 type methods + // ------------------------------------------------------------------------- + + @Override + public Row9 fieldsRow() { + return (Row9) super.fieldsRow(); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/WorkflowRuntimeStatistics.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/WorkflowRuntimeStatistics.java deleted file mode 100644 index c9435def77b..00000000000 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/WorkflowRuntimeStatistics.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * This file is generated by jOOQ. - */ -package edu.uci.ics.texera.dao.jooq.generated.tables; - - -import edu.uci.ics.texera.dao.jooq.generated.Indexes; -import edu.uci.ics.texera.dao.jooq.generated.Keys; -import edu.uci.ics.texera.dao.jooq.generated.TexeraDb; -import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowRuntimeStatisticsRecord; - -import java.sql.Timestamp; -import java.util.Arrays; -import java.util.List; - -import org.jooq.Field; -import org.jooq.ForeignKey; -import org.jooq.Index; -import org.jooq.Name; -import org.jooq.Record; -import org.jooq.Row11; -import org.jooq.Schema; -import org.jooq.Table; -import org.jooq.TableField; -import org.jooq.UniqueKey; -import org.jooq.impl.DSL; -import org.jooq.impl.TableImpl; -import org.jooq.types.UInteger; -import org.jooq.types.ULong; - - -/** - * This class is generated by jOOQ. - */ -@SuppressWarnings({ "all", "unchecked", "rawtypes" }) -public class WorkflowRuntimeStatistics extends TableImpl { - - private static final long serialVersionUID = -327153807; - - /** - * The reference instance of texera_db.workflow_runtime_statistics - */ - public static final WorkflowRuntimeStatistics WORKFLOW_RUNTIME_STATISTICS = new WorkflowRuntimeStatistics(); - - /** - * The class holding records for this type - */ - @Override - public Class getRecordType() { - return WorkflowRuntimeStatisticsRecord.class; - } - - /** - * The column texera_db.workflow_runtime_statistics.workflow_id. - */ - public final TableField WORKFLOW_ID = createField(DSL.name("workflow_id"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.execution_id. - */ - public final TableField EXECUTION_ID = createField(DSL.name("execution_id"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.operator_id. - */ - public final TableField OPERATOR_ID = createField(DSL.name("operator_id"), org.jooq.impl.SQLDataType.VARCHAR(100).nullable(false), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.time. - */ - public final TableField TIME = createField(DSL.name("time"), org.jooq.impl.SQLDataType.TIMESTAMP.nullable(false).defaultValue(org.jooq.impl.DSL.field("CURRENT_TIMESTAMP(6)", org.jooq.impl.SQLDataType.TIMESTAMP)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.input_tuple_cnt. - */ - public final TableField INPUT_TUPLE_CNT = createField(DSL.name("input_tuple_cnt"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.INTEGERUNSIGNED)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.output_tuple_cnt. - */ - public final TableField OUTPUT_TUPLE_CNT = createField(DSL.name("output_tuple_cnt"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.INTEGERUNSIGNED)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.status. - */ - public final TableField STATUS = createField(DSL.name("status"), org.jooq.impl.SQLDataType.TINYINT.nullable(false).defaultValue(org.jooq.impl.DSL.inline("1", org.jooq.impl.SQLDataType.TINYINT)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.data_processing_time. - */ - public final TableField DATA_PROCESSING_TIME = createField(DSL.name("data_processing_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.control_processing_time. - */ - public final TableField CONTROL_PROCESSING_TIME = createField(DSL.name("control_processing_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.idle_time. - */ - public final TableField IDLE_TIME = createField(DSL.name("idle_time"), org.jooq.impl.SQLDataType.BIGINTUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.BIGINTUNSIGNED)), this, ""); - - /** - * The column texera_db.workflow_runtime_statistics.num_workers. - */ - public final TableField NUM_WORKERS = createField(DSL.name("num_workers"), org.jooq.impl.SQLDataType.INTEGERUNSIGNED.nullable(false).defaultValue(org.jooq.impl.DSL.inline("0", org.jooq.impl.SQLDataType.INTEGERUNSIGNED)), this, ""); - - /** - * Create a texera_db.workflow_runtime_statistics table reference - */ - public WorkflowRuntimeStatistics() { - this(DSL.name("workflow_runtime_statistics"), null); - } - - /** - * Create an aliased texera_db.workflow_runtime_statistics table reference - */ - public WorkflowRuntimeStatistics(String alias) { - this(DSL.name(alias), WORKFLOW_RUNTIME_STATISTICS); - } - - /** - * Create an aliased texera_db.workflow_runtime_statistics table reference - */ - public WorkflowRuntimeStatistics(Name alias) { - this(alias, WORKFLOW_RUNTIME_STATISTICS); - } - - private WorkflowRuntimeStatistics(Name alias, Table aliased) { - this(alias, aliased, null); - } - - private WorkflowRuntimeStatistics(Name alias, Table aliased, Field[] parameters) { - super(alias, null, aliased, parameters, DSL.comment("")); - } - - public WorkflowRuntimeStatistics(Table child, ForeignKey key) { - super(child, key, WORKFLOW_RUNTIME_STATISTICS); - } - - @Override - public Schema getSchema() { - return TexeraDb.TEXERA_DB; - } - - @Override - public List getIndexes() { - return Arrays.asList(Indexes.WORKFLOW_RUNTIME_STATISTICS_EXECUTION_ID, Indexes.WORKFLOW_RUNTIME_STATISTICS_PRIMARY); - } - - @Override - public UniqueKey getPrimaryKey() { - return Keys.KEY_WORKFLOW_RUNTIME_STATISTICS_PRIMARY; - } - - @Override - public List> getKeys() { - return Arrays.>asList(Keys.KEY_WORKFLOW_RUNTIME_STATISTICS_PRIMARY); - } - - @Override - public List> getReferences() { - return Arrays.>asList(Keys.WORKFLOW_RUNTIME_STATISTICS_IBFK_1, Keys.WORKFLOW_RUNTIME_STATISTICS_IBFK_2); - } - - public Workflow workflow() { - return new Workflow(this, Keys.WORKFLOW_RUNTIME_STATISTICS_IBFK_1); - } - - public WorkflowExecutions workflowExecutions() { - return new WorkflowExecutions(this, Keys.WORKFLOW_RUNTIME_STATISTICS_IBFK_2); - } - - @Override - public WorkflowRuntimeStatistics as(String alias) { - return new WorkflowRuntimeStatistics(DSL.name(alias), this); - } - - @Override - public WorkflowRuntimeStatistics as(Name alias) { - return new WorkflowRuntimeStatistics(alias, this); - } - - /** - * Rename this table - */ - @Override - public WorkflowRuntimeStatistics rename(String name) { - return new WorkflowRuntimeStatistics(DSL.name(name), null); - } - - /** - * Rename this table - */ - @Override - public WorkflowRuntimeStatistics rename(Name name) { - return new WorkflowRuntimeStatistics(name, null); - } - - // ------------------------------------------------------------------------- - // Row11 type methods - // ------------------------------------------------------------------------- - - @Override - public Row11 fieldsRow() { - return (Row11) super.fieldsRow(); - } -} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorExecutionsDao.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorExecutionsDao.java new file mode 100644 index 00000000000..cb63b6b84b5 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorExecutionsDao.java @@ -0,0 +1,91 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.daos; + + +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorExecutionsRecord; + +import java.util.List; + +import org.jooq.Configuration; +import org.jooq.impl.DAOImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorExecutionsDao extends DAOImpl { + + /** + * Create a new OperatorExecutionsDao without any configuration + */ + public OperatorExecutionsDao() { + super(OperatorExecutions.OPERATOR_EXECUTIONS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorExecutions.class); + } + + /** + * Create a new OperatorExecutionsDao with an attached configuration + */ + public OperatorExecutionsDao(Configuration configuration) { + super(OperatorExecutions.OPERATOR_EXECUTIONS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorExecutions.class, configuration); + } + + @Override + public ULong getId(edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorExecutions object) { + return object.getOperatorExecutionId(); + } + + /** + * Fetch records that have operator_execution_id BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfOperatorExecutionId(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have operator_execution_id IN (values) + */ + public List fetchByOperatorExecutionId(ULong... values) { + return fetch(OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID, values); + } + + /** + * Fetch a unique record that has operator_execution_id = value + */ + public edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorExecutions fetchOneByOperatorExecutionId(ULong value) { + return fetchOne(OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID, value); + } + + /** + * Fetch records that have workflow_execution_id BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfWorkflowExecutionId(UInteger lowerInclusive, UInteger upperInclusive) { + return fetchRange(OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have workflow_execution_id IN (values) + */ + public List fetchByWorkflowExecutionId(UInteger... values) { + return fetch(OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID, values); + } + + /** + * Fetch records that have operator_id BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfOperatorId(String lowerInclusive, String upperInclusive) { + return fetchRange(OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_ID, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have operator_id IN (values) + */ + public List fetchByOperatorId(String... values) { + return fetch(OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_ID, values); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorRuntimeStatisticsDao.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorRuntimeStatisticsDao.java new file mode 100644 index 00000000000..8f7a79d5aa6 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/OperatorRuntimeStatisticsDao.java @@ -0,0 +1,170 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.daos; + + +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; +import edu.uci.ics.texera.dao.jooq.generated.tables.records.OperatorRuntimeStatisticsRecord; + +import java.sql.Timestamp; +import java.util.List; + +import org.jooq.Configuration; +import org.jooq.Record2; +import org.jooq.impl.DAOImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorRuntimeStatisticsDao extends DAOImpl> { + + /** + * Create a new OperatorRuntimeStatisticsDao without any configuration + */ + public OperatorRuntimeStatisticsDao() { + super(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorRuntimeStatistics.class); + } + + /** + * Create a new OperatorRuntimeStatisticsDao with an attached configuration + */ + public OperatorRuntimeStatisticsDao(Configuration configuration) { + super(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorRuntimeStatistics.class, configuration); + } + + @Override + public Record2 getId(edu.uci.ics.texera.dao.jooq.generated.tables.pojos.OperatorRuntimeStatistics object) { + return compositeKeyRecord(object.getOperatorExecutionId(), object.getTime()); + } + + /** + * Fetch records that have operator_execution_id BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfOperatorExecutionId(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have operator_execution_id IN (values) + */ + public List fetchByOperatorExecutionId(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID, values); + } + + /** + * Fetch records that have time BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfTime(Timestamp lowerInclusive, Timestamp upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.TIME, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have time IN (values) + */ + public List fetchByTime(Timestamp... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.TIME, values); + } + + /** + * Fetch records that have input_tuple_cnt BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfInputTupleCnt(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have input_tuple_cnt IN (values) + */ + public List fetchByInputTupleCnt(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, values); + } + + /** + * Fetch records that have output_tuple_cnt BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfOutputTupleCnt(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have output_tuple_cnt IN (values) + */ + public List fetchByOutputTupleCnt(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, values); + } + + /** + * Fetch records that have status BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfStatus(Byte lowerInclusive, Byte upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.STATUS, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have status IN (values) + */ + public List fetchByStatus(Byte... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.STATUS, values); + } + + /** + * Fetch records that have data_processing_time BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfDataProcessingTime(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have data_processing_time IN (values) + */ + public List fetchByDataProcessingTime(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, values); + } + + /** + * Fetch records that have control_processing_time BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfControlProcessingTime(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have control_processing_time IN (values) + */ + public List fetchByControlProcessingTime(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, values); + } + + /** + * Fetch records that have idle_time BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfIdleTime(ULong lowerInclusive, ULong upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.IDLE_TIME, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have idle_time IN (values) + */ + public List fetchByIdleTime(ULong... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.IDLE_TIME, values); + } + + /** + * Fetch records that have num_workers BETWEEN lowerInclusive AND upperInclusive + */ + public List fetchRangeOfNumWorkers(UInteger lowerInclusive, UInteger upperInclusive) { + return fetchRange(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.NUM_WORKERS, lowerInclusive, upperInclusive); + } + + /** + * Fetch records that have num_workers IN (values) + */ + public List fetchByNumWorkers(UInteger... values) { + return fetch(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.NUM_WORKERS, values); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/WorkflowRuntimeStatisticsDao.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/WorkflowRuntimeStatisticsDao.java deleted file mode 100644 index 7db23db91cb..00000000000 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/daos/WorkflowRuntimeStatisticsDao.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * This file is generated by jOOQ. - */ -package edu.uci.ics.texera.dao.jooq.generated.tables.daos; - - -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; -import edu.uci.ics.texera.dao.jooq.generated.tables.records.WorkflowRuntimeStatisticsRecord; - -import java.sql.Timestamp; -import java.util.List; - -import org.jooq.Configuration; -import org.jooq.Record4; -import org.jooq.impl.DAOImpl; -import org.jooq.types.UInteger; -import org.jooq.types.ULong; - - -/** - * This class is generated by jOOQ. - */ -@SuppressWarnings({ "all", "unchecked", "rawtypes" }) -public class WorkflowRuntimeStatisticsDao extends DAOImpl> { - - /** - * Create a new WorkflowRuntimeStatisticsDao without any configuration - */ - public WorkflowRuntimeStatisticsDao() { - super(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.WorkflowRuntimeStatistics.class); - } - - /** - * Create a new WorkflowRuntimeStatisticsDao with an attached configuration - */ - public WorkflowRuntimeStatisticsDao(Configuration configuration) { - super(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS, edu.uci.ics.texera.dao.jooq.generated.tables.pojos.WorkflowRuntimeStatistics.class, configuration); - } - - @Override - public Record4 getId(edu.uci.ics.texera.dao.jooq.generated.tables.pojos.WorkflowRuntimeStatistics object) { - return compositeKeyRecord(object.getWorkflowId(), object.getExecutionId(), object.getOperatorId(), object.getTime()); - } - - /** - * Fetch records that have workflow_id BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfWorkflowId(UInteger lowerInclusive, UInteger upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have workflow_id IN (values) - */ - public List fetchByWorkflowId(UInteger... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID, values); - } - - /** - * Fetch records that have execution_id BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfExecutionId(UInteger lowerInclusive, UInteger upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have execution_id IN (values) - */ - public List fetchByExecutionId(UInteger... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID, values); - } - - /** - * Fetch records that have operator_id BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfOperatorId(String lowerInclusive, String upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have operator_id IN (values) - */ - public List fetchByOperatorId(String... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID, values); - } - - /** - * Fetch records that have time BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfTime(Timestamp lowerInclusive, Timestamp upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.TIME, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have time IN (values) - */ - public List fetchByTime(Timestamp... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.TIME, values); - } - - /** - * Fetch records that have input_tuple_cnt BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfInputTupleCnt(UInteger lowerInclusive, UInteger upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have input_tuple_cnt IN (values) - */ - public List fetchByInputTupleCnt(UInteger... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.INPUT_TUPLE_CNT, values); - } - - /** - * Fetch records that have output_tuple_cnt BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfOutputTupleCnt(UInteger lowerInclusive, UInteger upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have output_tuple_cnt IN (values) - */ - public List fetchByOutputTupleCnt(UInteger... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT, values); - } - - /** - * Fetch records that have status BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfStatus(Byte lowerInclusive, Byte upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.STATUS, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have status IN (values) - */ - public List fetchByStatus(Byte... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.STATUS, values); - } - - /** - * Fetch records that have data_processing_time BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfDataProcessingTime(ULong lowerInclusive, ULong upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have data_processing_time IN (values) - */ - public List fetchByDataProcessingTime(ULong... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.DATA_PROCESSING_TIME, values); - } - - /** - * Fetch records that have control_processing_time BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfControlProcessingTime(ULong lowerInclusive, ULong upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have control_processing_time IN (values) - */ - public List fetchByControlProcessingTime(ULong... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME, values); - } - - /** - * Fetch records that have idle_time BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfIdleTime(ULong lowerInclusive, ULong upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.IDLE_TIME, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have idle_time IN (values) - */ - public List fetchByIdleTime(ULong... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.IDLE_TIME, values); - } - - /** - * Fetch records that have num_workers BETWEEN lowerInclusive AND upperInclusive - */ - public List fetchRangeOfNumWorkers(UInteger lowerInclusive, UInteger upperInclusive) { - return fetchRange(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.NUM_WORKERS, lowerInclusive, upperInclusive); - } - - /** - * Fetch records that have num_workers IN (values) - */ - public List fetchByNumWorkers(UInteger... values) { - return fetch(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.NUM_WORKERS, values); - } -} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorExecutions.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorExecutions.java new file mode 100644 index 00000000000..c7582ef90e3 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorExecutions.java @@ -0,0 +1,62 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.interfaces; + + +import java.io.Serializable; + +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public interface IOperatorExecutions extends Serializable { + + /** + * Setter for texera_db.operator_executions.operator_execution_id. + */ + public void setOperatorExecutionId(ULong value); + + /** + * Getter for texera_db.operator_executions.operator_execution_id. + */ + public ULong getOperatorExecutionId(); + + /** + * Setter for texera_db.operator_executions.workflow_execution_id. + */ + public void setWorkflowExecutionId(UInteger value); + + /** + * Getter for texera_db.operator_executions.workflow_execution_id. + */ + public UInteger getWorkflowExecutionId(); + + /** + * Setter for texera_db.operator_executions.operator_id. + */ + public void setOperatorId(String value); + + /** + * Getter for texera_db.operator_executions.operator_id. + */ + public String getOperatorId(); + + // ------------------------------------------------------------------------- + // FROM and INTO + // ------------------------------------------------------------------------- + + /** + * Load data from another generated Record/POJO implementing the common interface IOperatorExecutions + */ + public void from(edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorExecutions from); + + /** + * Copy data into another generated Record/POJO implementing the common interface IOperatorExecutions + */ + public E into(E into); +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorRuntimeStatistics.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorRuntimeStatistics.java new file mode 100644 index 00000000000..b499211e442 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IOperatorRuntimeStatistics.java @@ -0,0 +1,123 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.interfaces; + + +import java.io.Serializable; +import java.sql.Timestamp; + +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public interface IOperatorRuntimeStatistics extends Serializable { + + /** + * Setter for texera_db.operator_runtime_statistics.operator_execution_id. + */ + public void setOperatorExecutionId(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.operator_execution_id. + */ + public ULong getOperatorExecutionId(); + + /** + * Setter for texera_db.operator_runtime_statistics.time. + */ + public void setTime(Timestamp value); + + /** + * Getter for texera_db.operator_runtime_statistics.time. + */ + public Timestamp getTime(); + + /** + * Setter for texera_db.operator_runtime_statistics.input_tuple_cnt. + */ + public void setInputTupleCnt(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.input_tuple_cnt. + */ + public ULong getInputTupleCnt(); + + /** + * Setter for texera_db.operator_runtime_statistics.output_tuple_cnt. + */ + public void setOutputTupleCnt(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.output_tuple_cnt. + */ + public ULong getOutputTupleCnt(); + + /** + * Setter for texera_db.operator_runtime_statistics.status. + */ + public void setStatus(Byte value); + + /** + * Getter for texera_db.operator_runtime_statistics.status. + */ + public Byte getStatus(); + + /** + * Setter for texera_db.operator_runtime_statistics.data_processing_time. + */ + public void setDataProcessingTime(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.data_processing_time. + */ + public ULong getDataProcessingTime(); + + /** + * Setter for texera_db.operator_runtime_statistics.control_processing_time. + */ + public void setControlProcessingTime(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.control_processing_time. + */ + public ULong getControlProcessingTime(); + + /** + * Setter for texera_db.operator_runtime_statistics.idle_time. + */ + public void setIdleTime(ULong value); + + /** + * Getter for texera_db.operator_runtime_statistics.idle_time. + */ + public ULong getIdleTime(); + + /** + * Setter for texera_db.operator_runtime_statistics.num_workers. + */ + public void setNumWorkers(UInteger value); + + /** + * Getter for texera_db.operator_runtime_statistics.num_workers. + */ + public UInteger getNumWorkers(); + + // ------------------------------------------------------------------------- + // FROM and INTO + // ------------------------------------------------------------------------- + + /** + * Load data from another generated Record/POJO implementing the common interface IOperatorRuntimeStatistics + */ + public void from(edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorRuntimeStatistics from); + + /** + * Copy data into another generated Record/POJO implementing the common interface IOperatorRuntimeStatistics + */ + public E into(E into); +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IWorkflowRuntimeStatistics.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IWorkflowRuntimeStatistics.java deleted file mode 100644 index 17c34923196..00000000000 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/interfaces/IWorkflowRuntimeStatistics.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * This file is generated by jOOQ. - */ -package edu.uci.ics.texera.dao.jooq.generated.tables.interfaces; - - -import java.io.Serializable; -import java.sql.Timestamp; - -import org.jooq.types.UInteger; -import org.jooq.types.ULong; - - -/** - * This class is generated by jOOQ. - */ -@SuppressWarnings({ "all", "unchecked", "rawtypes" }) -public interface IWorkflowRuntimeStatistics extends Serializable { - - /** - * Setter for texera_db.workflow_runtime_statistics.workflow_id. - */ - public void setWorkflowId(UInteger value); - - /** - * Getter for texera_db.workflow_runtime_statistics.workflow_id. - */ - public UInteger getWorkflowId(); - - /** - * Setter for texera_db.workflow_runtime_statistics.execution_id. - */ - public void setExecutionId(UInteger value); - - /** - * Getter for texera_db.workflow_runtime_statistics.execution_id. - */ - public UInteger getExecutionId(); - - /** - * Setter for texera_db.workflow_runtime_statistics.operator_id. - */ - public void setOperatorId(String value); - - /** - * Getter for texera_db.workflow_runtime_statistics.operator_id. - */ - public String getOperatorId(); - - /** - * Setter for texera_db.workflow_runtime_statistics.time. - */ - public void setTime(Timestamp value); - - /** - * Getter for texera_db.workflow_runtime_statistics.time. - */ - public Timestamp getTime(); - - /** - * Setter for texera_db.workflow_runtime_statistics.input_tuple_cnt. - */ - public void setInputTupleCnt(UInteger value); - - /** - * Getter for texera_db.workflow_runtime_statistics.input_tuple_cnt. - */ - public UInteger getInputTupleCnt(); - - /** - * Setter for texera_db.workflow_runtime_statistics.output_tuple_cnt. - */ - public void setOutputTupleCnt(UInteger value); - - /** - * Getter for texera_db.workflow_runtime_statistics.output_tuple_cnt. - */ - public UInteger getOutputTupleCnt(); - - /** - * Setter for texera_db.workflow_runtime_statistics.status. - */ - public void setStatus(Byte value); - - /** - * Getter for texera_db.workflow_runtime_statistics.status. - */ - public Byte getStatus(); - - /** - * Setter for texera_db.workflow_runtime_statistics.data_processing_time. - */ - public void setDataProcessingTime(ULong value); - - /** - * Getter for texera_db.workflow_runtime_statistics.data_processing_time. - */ - public ULong getDataProcessingTime(); - - /** - * Setter for texera_db.workflow_runtime_statistics.control_processing_time. - */ - public void setControlProcessingTime(ULong value); - - /** - * Getter for texera_db.workflow_runtime_statistics.control_processing_time. - */ - public ULong getControlProcessingTime(); - - /** - * Setter for texera_db.workflow_runtime_statistics.idle_time. - */ - public void setIdleTime(ULong value); - - /** - * Getter for texera_db.workflow_runtime_statistics.idle_time. - */ - public ULong getIdleTime(); - - /** - * Setter for texera_db.workflow_runtime_statistics.num_workers. - */ - public void setNumWorkers(UInteger value); - - /** - * Getter for texera_db.workflow_runtime_statistics.num_workers. - */ - public UInteger getNumWorkers(); - - // ------------------------------------------------------------------------- - // FROM and INTO - // ------------------------------------------------------------------------- - - /** - * Load data from another generated Record/POJO implementing the common interface IWorkflowRuntimeStatistics - */ - public void from(edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IWorkflowRuntimeStatistics from); - - /** - * Copy data into another generated Record/POJO implementing the common interface IWorkflowRuntimeStatistics - */ - public E into(E into); -} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorExecutions.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorExecutions.java new file mode 100644 index 00000000000..4d7b40c2db1 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorExecutions.java @@ -0,0 +1,101 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.pojos; + + +import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorExecutions; + +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorExecutions implements IOperatorExecutions { + + private static final long serialVersionUID = -1568831818; + + private ULong operatorExecutionId; + private UInteger workflowExecutionId; + private String operatorId; + + public OperatorExecutions() {} + + public OperatorExecutions(IOperatorExecutions value) { + this.operatorExecutionId = value.getOperatorExecutionId(); + this.workflowExecutionId = value.getWorkflowExecutionId(); + this.operatorId = value.getOperatorId(); + } + + public OperatorExecutions( + ULong operatorExecutionId, + UInteger workflowExecutionId, + String operatorId + ) { + this.operatorExecutionId = operatorExecutionId; + this.workflowExecutionId = workflowExecutionId; + this.operatorId = operatorId; + } + + @Override + public ULong getOperatorExecutionId() { + return this.operatorExecutionId; + } + + @Override + public void setOperatorExecutionId(ULong operatorExecutionId) { + this.operatorExecutionId = operatorExecutionId; + } + + @Override + public UInteger getWorkflowExecutionId() { + return this.workflowExecutionId; + } + + @Override + public void setWorkflowExecutionId(UInteger workflowExecutionId) { + this.workflowExecutionId = workflowExecutionId; + } + + @Override + public String getOperatorId() { + return this.operatorId; + } + + @Override + public void setOperatorId(String operatorId) { + this.operatorId = operatorId; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OperatorExecutions ("); + + sb.append(operatorExecutionId); + sb.append(", ").append(workflowExecutionId); + sb.append(", ").append(operatorId); + + sb.append(")"); + return sb.toString(); + } + + // ------------------------------------------------------------------------- + // FROM and INTO + // ------------------------------------------------------------------------- + + @Override + public void from(IOperatorExecutions from) { + setOperatorExecutionId(from.getOperatorExecutionId()); + setWorkflowExecutionId(from.getWorkflowExecutionId()); + setOperatorId(from.getOperatorId()); + } + + @Override + public E into(E into) { + into.from(this); + return into; + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/WorkflowRuntimeStatistics.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorRuntimeStatistics.java similarity index 66% rename from core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/WorkflowRuntimeStatistics.java rename to core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorRuntimeStatistics.java index f72f5a10179..f48c3df1046 100644 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/WorkflowRuntimeStatistics.java +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/pojos/OperatorRuntimeStatistics.java @@ -4,7 +4,7 @@ package edu.uci.ics.texera.dao.jooq.generated.tables.pojos; -import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IWorkflowRuntimeStatistics; +import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorRuntimeStatistics; import java.sql.Timestamp; @@ -16,28 +16,24 @@ * This class is generated by jOOQ. */ @SuppressWarnings({ "all", "unchecked", "rawtypes" }) -public class WorkflowRuntimeStatistics implements IWorkflowRuntimeStatistics { +public class OperatorRuntimeStatistics implements IOperatorRuntimeStatistics { - private static final long serialVersionUID = 511530419; + private static final long serialVersionUID = -175115514; - private UInteger workflowId; - private UInteger executionId; - private String operatorId; + private ULong operatorExecutionId; private Timestamp time; - private UInteger inputTupleCnt; - private UInteger outputTupleCnt; + private ULong inputTupleCnt; + private ULong outputTupleCnt; private Byte status; private ULong dataProcessingTime; private ULong controlProcessingTime; private ULong idleTime; private UInteger numWorkers; - public WorkflowRuntimeStatistics() {} + public OperatorRuntimeStatistics() {} - public WorkflowRuntimeStatistics(IWorkflowRuntimeStatistics value) { - this.workflowId = value.getWorkflowId(); - this.executionId = value.getExecutionId(); - this.operatorId = value.getOperatorId(); + public OperatorRuntimeStatistics(IOperatorRuntimeStatistics value) { + this.operatorExecutionId = value.getOperatorExecutionId(); this.time = value.getTime(); this.inputTupleCnt = value.getInputTupleCnt(); this.outputTupleCnt = value.getOutputTupleCnt(); @@ -48,22 +44,18 @@ public WorkflowRuntimeStatistics(IWorkflowRuntimeStatistics value) { this.numWorkers = value.getNumWorkers(); } - public WorkflowRuntimeStatistics( - UInteger workflowId, - UInteger executionId, - String operatorId, + public OperatorRuntimeStatistics( + ULong operatorExecutionId, Timestamp time, - UInteger inputTupleCnt, - UInteger outputTupleCnt, + ULong inputTupleCnt, + ULong outputTupleCnt, Byte status, ULong dataProcessingTime, ULong controlProcessingTime, ULong idleTime, UInteger numWorkers ) { - this.workflowId = workflowId; - this.executionId = executionId; - this.operatorId = operatorId; + this.operatorExecutionId = operatorExecutionId; this.time = time; this.inputTupleCnt = inputTupleCnt; this.outputTupleCnt = outputTupleCnt; @@ -75,33 +67,13 @@ public WorkflowRuntimeStatistics( } @Override - public UInteger getWorkflowId() { - return this.workflowId; + public ULong getOperatorExecutionId() { + return this.operatorExecutionId; } @Override - public void setWorkflowId(UInteger workflowId) { - this.workflowId = workflowId; - } - - @Override - public UInteger getExecutionId() { - return this.executionId; - } - - @Override - public void setExecutionId(UInteger executionId) { - this.executionId = executionId; - } - - @Override - public String getOperatorId() { - return this.operatorId; - } - - @Override - public void setOperatorId(String operatorId) { - this.operatorId = operatorId; + public void setOperatorExecutionId(ULong operatorExecutionId) { + this.operatorExecutionId = operatorExecutionId; } @Override @@ -115,22 +87,22 @@ public void setTime(Timestamp time) { } @Override - public UInteger getInputTupleCnt() { + public ULong getInputTupleCnt() { return this.inputTupleCnt; } @Override - public void setInputTupleCnt(UInteger inputTupleCnt) { + public void setInputTupleCnt(ULong inputTupleCnt) { this.inputTupleCnt = inputTupleCnt; } @Override - public UInteger getOutputTupleCnt() { + public ULong getOutputTupleCnt() { return this.outputTupleCnt; } @Override - public void setOutputTupleCnt(UInteger outputTupleCnt) { + public void setOutputTupleCnt(ULong outputTupleCnt) { this.outputTupleCnt = outputTupleCnt; } @@ -186,11 +158,9 @@ public void setNumWorkers(UInteger numWorkers) { @Override public String toString() { - StringBuilder sb = new StringBuilder("WorkflowRuntimeStatistics ("); + StringBuilder sb = new StringBuilder("OperatorRuntimeStatistics ("); - sb.append(workflowId); - sb.append(", ").append(executionId); - sb.append(", ").append(operatorId); + sb.append(operatorExecutionId); sb.append(", ").append(time); sb.append(", ").append(inputTupleCnt); sb.append(", ").append(outputTupleCnt); @@ -209,10 +179,8 @@ public String toString() { // ------------------------------------------------------------------------- @Override - public void from(IWorkflowRuntimeStatistics from) { - setWorkflowId(from.getWorkflowId()); - setExecutionId(from.getExecutionId()); - setOperatorId(from.getOperatorId()); + public void from(IOperatorRuntimeStatistics from) { + setOperatorExecutionId(from.getOperatorExecutionId()); setTime(from.getTime()); setInputTupleCnt(from.getInputTupleCnt()); setOutputTupleCnt(from.getOutputTupleCnt()); @@ -224,7 +192,7 @@ public void from(IWorkflowRuntimeStatistics from) { } @Override - public E into(E into) { + public E into(E into) { into.from(this); return into; } diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorExecutionsRecord.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorExecutionsRecord.java new file mode 100644 index 00000000000..f3b78e48a4e --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorExecutionsRecord.java @@ -0,0 +1,207 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.records; + + +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorExecutions; +import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorExecutions; + +import org.jooq.Field; +import org.jooq.Record1; +import org.jooq.Record3; +import org.jooq.Row3; +import org.jooq.impl.UpdatableRecordImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorExecutionsRecord extends UpdatableRecordImpl implements Record3, IOperatorExecutions { + + private static final long serialVersionUID = 1433337880; + + /** + * Setter for texera_db.operator_executions.operator_execution_id. + */ + @Override + public void setOperatorExecutionId(ULong value) { + set(0, value); + } + + /** + * Getter for texera_db.operator_executions.operator_execution_id. + */ + @Override + public ULong getOperatorExecutionId() { + return (ULong) get(0); + } + + /** + * Setter for texera_db.operator_executions.workflow_execution_id. + */ + @Override + public void setWorkflowExecutionId(UInteger value) { + set(1, value); + } + + /** + * Getter for texera_db.operator_executions.workflow_execution_id. + */ + @Override + public UInteger getWorkflowExecutionId() { + return (UInteger) get(1); + } + + /** + * Setter for texera_db.operator_executions.operator_id. + */ + @Override + public void setOperatorId(String value) { + set(2, value); + } + + /** + * Getter for texera_db.operator_executions.operator_id. + */ + @Override + public String getOperatorId() { + return (String) get(2); + } + + // ------------------------------------------------------------------------- + // Primary key information + // ------------------------------------------------------------------------- + + @Override + public Record1 key() { + return (Record1) super.key(); + } + + // ------------------------------------------------------------------------- + // Record3 type implementation + // ------------------------------------------------------------------------- + + @Override + public Row3 fieldsRow() { + return (Row3) super.fieldsRow(); + } + + @Override + public Row3 valuesRow() { + return (Row3) super.valuesRow(); + } + + @Override + public Field field1() { + return OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_EXECUTION_ID; + } + + @Override + public Field field2() { + return OperatorExecutions.OPERATOR_EXECUTIONS.WORKFLOW_EXECUTION_ID; + } + + @Override + public Field field3() { + return OperatorExecutions.OPERATOR_EXECUTIONS.OPERATOR_ID; + } + + @Override + public ULong component1() { + return getOperatorExecutionId(); + } + + @Override + public UInteger component2() { + return getWorkflowExecutionId(); + } + + @Override + public String component3() { + return getOperatorId(); + } + + @Override + public ULong value1() { + return getOperatorExecutionId(); + } + + @Override + public UInteger value2() { + return getWorkflowExecutionId(); + } + + @Override + public String value3() { + return getOperatorId(); + } + + @Override + public OperatorExecutionsRecord value1(ULong value) { + setOperatorExecutionId(value); + return this; + } + + @Override + public OperatorExecutionsRecord value2(UInteger value) { + setWorkflowExecutionId(value); + return this; + } + + @Override + public OperatorExecutionsRecord value3(String value) { + setOperatorId(value); + return this; + } + + @Override + public OperatorExecutionsRecord values(ULong value1, UInteger value2, String value3) { + value1(value1); + value2(value2); + value3(value3); + return this; + } + + // ------------------------------------------------------------------------- + // FROM and INTO + // ------------------------------------------------------------------------- + + @Override + public void from(IOperatorExecutions from) { + setOperatorExecutionId(from.getOperatorExecutionId()); + setWorkflowExecutionId(from.getWorkflowExecutionId()); + setOperatorId(from.getOperatorId()); + } + + @Override + public E into(E into) { + into.from(this); + return into; + } + + // ------------------------------------------------------------------------- + // Constructors + // ------------------------------------------------------------------------- + + /** + * Create a detached OperatorExecutionsRecord + */ + public OperatorExecutionsRecord() { + super(OperatorExecutions.OPERATOR_EXECUTIONS); + } + + /** + * Create a detached, initialised OperatorExecutionsRecord + */ + public OperatorExecutionsRecord(ULong operatorExecutionId, UInteger workflowExecutionId, String operatorId) { + super(OperatorExecutions.OPERATOR_EXECUTIONS); + + set(0, operatorExecutionId); + set(1, workflowExecutionId); + set(2, operatorId); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorRuntimeStatisticsRecord.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorRuntimeStatisticsRecord.java new file mode 100644 index 00000000000..f34ef3c1c43 --- /dev/null +++ b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/OperatorRuntimeStatisticsRecord.java @@ -0,0 +1,449 @@ +/* + * This file is generated by jOOQ. + */ +package edu.uci.ics.texera.dao.jooq.generated.tables.records; + + +import edu.uci.ics.texera.dao.jooq.generated.tables.OperatorRuntimeStatistics; +import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IOperatorRuntimeStatistics; + +import java.sql.Timestamp; + +import org.jooq.Field; +import org.jooq.Record2; +import org.jooq.Record9; +import org.jooq.Row9; +import org.jooq.impl.UpdatableRecordImpl; +import org.jooq.types.UInteger; +import org.jooq.types.ULong; + + +/** + * This class is generated by jOOQ. + */ +@SuppressWarnings({ "all", "unchecked", "rawtypes" }) +public class OperatorRuntimeStatisticsRecord extends UpdatableRecordImpl implements Record9, IOperatorRuntimeStatistics { + + private static final long serialVersionUID = -610841667; + + /** + * Setter for texera_db.operator_runtime_statistics.operator_execution_id. + */ + @Override + public void setOperatorExecutionId(ULong value) { + set(0, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.operator_execution_id. + */ + @Override + public ULong getOperatorExecutionId() { + return (ULong) get(0); + } + + /** + * Setter for texera_db.operator_runtime_statistics.time. + */ + @Override + public void setTime(Timestamp value) { + set(1, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.time. + */ + @Override + public Timestamp getTime() { + return (Timestamp) get(1); + } + + /** + * Setter for texera_db.operator_runtime_statistics.input_tuple_cnt. + */ + @Override + public void setInputTupleCnt(ULong value) { + set(2, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.input_tuple_cnt. + */ + @Override + public ULong getInputTupleCnt() { + return (ULong) get(2); + } + + /** + * Setter for texera_db.operator_runtime_statistics.output_tuple_cnt. + */ + @Override + public void setOutputTupleCnt(ULong value) { + set(3, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.output_tuple_cnt. + */ + @Override + public ULong getOutputTupleCnt() { + return (ULong) get(3); + } + + /** + * Setter for texera_db.operator_runtime_statistics.status. + */ + @Override + public void setStatus(Byte value) { + set(4, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.status. + */ + @Override + public Byte getStatus() { + return (Byte) get(4); + } + + /** + * Setter for texera_db.operator_runtime_statistics.data_processing_time. + */ + @Override + public void setDataProcessingTime(ULong value) { + set(5, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.data_processing_time. + */ + @Override + public ULong getDataProcessingTime() { + return (ULong) get(5); + } + + /** + * Setter for texera_db.operator_runtime_statistics.control_processing_time. + */ + @Override + public void setControlProcessingTime(ULong value) { + set(6, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.control_processing_time. + */ + @Override + public ULong getControlProcessingTime() { + return (ULong) get(6); + } + + /** + * Setter for texera_db.operator_runtime_statistics.idle_time. + */ + @Override + public void setIdleTime(ULong value) { + set(7, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.idle_time. + */ + @Override + public ULong getIdleTime() { + return (ULong) get(7); + } + + /** + * Setter for texera_db.operator_runtime_statistics.num_workers. + */ + @Override + public void setNumWorkers(UInteger value) { + set(8, value); + } + + /** + * Getter for texera_db.operator_runtime_statistics.num_workers. + */ + @Override + public UInteger getNumWorkers() { + return (UInteger) get(8); + } + + // ------------------------------------------------------------------------- + // Primary key information + // ------------------------------------------------------------------------- + + @Override + public Record2 key() { + return (Record2) super.key(); + } + + // ------------------------------------------------------------------------- + // Record9 type implementation + // ------------------------------------------------------------------------- + + @Override + public Row9 fieldsRow() { + return (Row9) super.fieldsRow(); + } + + @Override + public Row9 valuesRow() { + return (Row9) super.valuesRow(); + } + + @Override + public Field field1() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OPERATOR_EXECUTION_ID; + } + + @Override + public Field field2() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.TIME; + } + + @Override + public Field field3() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.INPUT_TUPLE_CNT; + } + + @Override + public Field field4() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT; + } + + @Override + public Field field5() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.STATUS; + } + + @Override + public Field field6() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.DATA_PROCESSING_TIME; + } + + @Override + public Field field7() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME; + } + + @Override + public Field field8() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.IDLE_TIME; + } + + @Override + public Field field9() { + return OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS.NUM_WORKERS; + } + + @Override + public ULong component1() { + return getOperatorExecutionId(); + } + + @Override + public Timestamp component2() { + return getTime(); + } + + @Override + public ULong component3() { + return getInputTupleCnt(); + } + + @Override + public ULong component4() { + return getOutputTupleCnt(); + } + + @Override + public Byte component5() { + return getStatus(); + } + + @Override + public ULong component6() { + return getDataProcessingTime(); + } + + @Override + public ULong component7() { + return getControlProcessingTime(); + } + + @Override + public ULong component8() { + return getIdleTime(); + } + + @Override + public UInteger component9() { + return getNumWorkers(); + } + + @Override + public ULong value1() { + return getOperatorExecutionId(); + } + + @Override + public Timestamp value2() { + return getTime(); + } + + @Override + public ULong value3() { + return getInputTupleCnt(); + } + + @Override + public ULong value4() { + return getOutputTupleCnt(); + } + + @Override + public Byte value5() { + return getStatus(); + } + + @Override + public ULong value6() { + return getDataProcessingTime(); + } + + @Override + public ULong value7() { + return getControlProcessingTime(); + } + + @Override + public ULong value8() { + return getIdleTime(); + } + + @Override + public UInteger value9() { + return getNumWorkers(); + } + + @Override + public OperatorRuntimeStatisticsRecord value1(ULong value) { + setOperatorExecutionId(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value2(Timestamp value) { + setTime(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value3(ULong value) { + setInputTupleCnt(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value4(ULong value) { + setOutputTupleCnt(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value5(Byte value) { + setStatus(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value6(ULong value) { + setDataProcessingTime(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value7(ULong value) { + setControlProcessingTime(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value8(ULong value) { + setIdleTime(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord value9(UInteger value) { + setNumWorkers(value); + return this; + } + + @Override + public OperatorRuntimeStatisticsRecord values(ULong value1, Timestamp value2, ULong value3, ULong value4, Byte value5, ULong value6, ULong value7, ULong value8, UInteger value9) { + value1(value1); + value2(value2); + value3(value3); + value4(value4); + value5(value5); + value6(value6); + value7(value7); + value8(value8); + value9(value9); + return this; + } + + // ------------------------------------------------------------------------- + // FROM and INTO + // ------------------------------------------------------------------------- + + @Override + public void from(IOperatorRuntimeStatistics from) { + setOperatorExecutionId(from.getOperatorExecutionId()); + setTime(from.getTime()); + setInputTupleCnt(from.getInputTupleCnt()); + setOutputTupleCnt(from.getOutputTupleCnt()); + setStatus(from.getStatus()); + setDataProcessingTime(from.getDataProcessingTime()); + setControlProcessingTime(from.getControlProcessingTime()); + setIdleTime(from.getIdleTime()); + setNumWorkers(from.getNumWorkers()); + } + + @Override + public E into(E into) { + into.from(this); + return into; + } + + // ------------------------------------------------------------------------- + // Constructors + // ------------------------------------------------------------------------- + + /** + * Create a detached OperatorRuntimeStatisticsRecord + */ + public OperatorRuntimeStatisticsRecord() { + super(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS); + } + + /** + * Create a detached, initialised OperatorRuntimeStatisticsRecord + */ + public OperatorRuntimeStatisticsRecord(ULong operatorExecutionId, Timestamp time, ULong inputTupleCnt, ULong outputTupleCnt, Byte status, ULong dataProcessingTime, ULong controlProcessingTime, ULong idleTime, UInteger numWorkers) { + super(OperatorRuntimeStatistics.OPERATOR_RUNTIME_STATISTICS); + + set(0, operatorExecutionId); + set(1, time); + set(2, inputTupleCnt); + set(3, outputTupleCnt); + set(4, status); + set(5, dataProcessingTime); + set(6, controlProcessingTime); + set(7, idleTime); + set(8, numWorkers); + } +} diff --git a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/WorkflowRuntimeStatisticsRecord.java b/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/WorkflowRuntimeStatisticsRecord.java deleted file mode 100644 index 34d16261444..00000000000 --- a/core/dao/src/main/scala/edu/uci/ics/texera/dao/jooq/generated/tables/records/WorkflowRuntimeStatisticsRecord.java +++ /dev/null @@ -1,529 +0,0 @@ -/* - * This file is generated by jOOQ. - */ -package edu.uci.ics.texera.dao.jooq.generated.tables.records; - - -import edu.uci.ics.texera.dao.jooq.generated.tables.WorkflowRuntimeStatistics; -import edu.uci.ics.texera.dao.jooq.generated.tables.interfaces.IWorkflowRuntimeStatistics; - -import java.sql.Timestamp; - -import org.jooq.Field; -import org.jooq.Record11; -import org.jooq.Record4; -import org.jooq.Row11; -import org.jooq.impl.UpdatableRecordImpl; -import org.jooq.types.UInteger; -import org.jooq.types.ULong; - - -/** - * This class is generated by jOOQ. - */ -@SuppressWarnings({ "all", "unchecked", "rawtypes" }) -public class WorkflowRuntimeStatisticsRecord extends UpdatableRecordImpl implements Record11, IWorkflowRuntimeStatistics { - - private static final long serialVersionUID = 101870722; - - /** - * Setter for texera_db.workflow_runtime_statistics.workflow_id. - */ - @Override - public void setWorkflowId(UInteger value) { - set(0, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.workflow_id. - */ - @Override - public UInteger getWorkflowId() { - return (UInteger) get(0); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.execution_id. - */ - @Override - public void setExecutionId(UInteger value) { - set(1, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.execution_id. - */ - @Override - public UInteger getExecutionId() { - return (UInteger) get(1); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.operator_id. - */ - @Override - public void setOperatorId(String value) { - set(2, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.operator_id. - */ - @Override - public String getOperatorId() { - return (String) get(2); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.time. - */ - @Override - public void setTime(Timestamp value) { - set(3, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.time. - */ - @Override - public Timestamp getTime() { - return (Timestamp) get(3); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.input_tuple_cnt. - */ - @Override - public void setInputTupleCnt(UInteger value) { - set(4, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.input_tuple_cnt. - */ - @Override - public UInteger getInputTupleCnt() { - return (UInteger) get(4); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.output_tuple_cnt. - */ - @Override - public void setOutputTupleCnt(UInteger value) { - set(5, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.output_tuple_cnt. - */ - @Override - public UInteger getOutputTupleCnt() { - return (UInteger) get(5); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.status. - */ - @Override - public void setStatus(Byte value) { - set(6, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.status. - */ - @Override - public Byte getStatus() { - return (Byte) get(6); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.data_processing_time. - */ - @Override - public void setDataProcessingTime(ULong value) { - set(7, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.data_processing_time. - */ - @Override - public ULong getDataProcessingTime() { - return (ULong) get(7); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.control_processing_time. - */ - @Override - public void setControlProcessingTime(ULong value) { - set(8, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.control_processing_time. - */ - @Override - public ULong getControlProcessingTime() { - return (ULong) get(8); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.idle_time. - */ - @Override - public void setIdleTime(ULong value) { - set(9, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.idle_time. - */ - @Override - public ULong getIdleTime() { - return (ULong) get(9); - } - - /** - * Setter for texera_db.workflow_runtime_statistics.num_workers. - */ - @Override - public void setNumWorkers(UInteger value) { - set(10, value); - } - - /** - * Getter for texera_db.workflow_runtime_statistics.num_workers. - */ - @Override - public UInteger getNumWorkers() { - return (UInteger) get(10); - } - - // ------------------------------------------------------------------------- - // Primary key information - // ------------------------------------------------------------------------- - - @Override - public Record4 key() { - return (Record4) super.key(); - } - - // ------------------------------------------------------------------------- - // Record11 type implementation - // ------------------------------------------------------------------------- - - @Override - public Row11 fieldsRow() { - return (Row11) super.fieldsRow(); - } - - @Override - public Row11 valuesRow() { - return (Row11) super.valuesRow(); - } - - @Override - public Field field1() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.WORKFLOW_ID; - } - - @Override - public Field field2() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.EXECUTION_ID; - } - - @Override - public Field field3() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OPERATOR_ID; - } - - @Override - public Field field4() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.TIME; - } - - @Override - public Field field5() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.INPUT_TUPLE_CNT; - } - - @Override - public Field field6() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.OUTPUT_TUPLE_CNT; - } - - @Override - public Field field7() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.STATUS; - } - - @Override - public Field field8() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.DATA_PROCESSING_TIME; - } - - @Override - public Field field9() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.CONTROL_PROCESSING_TIME; - } - - @Override - public Field field10() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.IDLE_TIME; - } - - @Override - public Field field11() { - return WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS.NUM_WORKERS; - } - - @Override - public UInteger component1() { - return getWorkflowId(); - } - - @Override - public UInteger component2() { - return getExecutionId(); - } - - @Override - public String component3() { - return getOperatorId(); - } - - @Override - public Timestamp component4() { - return getTime(); - } - - @Override - public UInteger component5() { - return getInputTupleCnt(); - } - - @Override - public UInteger component6() { - return getOutputTupleCnt(); - } - - @Override - public Byte component7() { - return getStatus(); - } - - @Override - public ULong component8() { - return getDataProcessingTime(); - } - - @Override - public ULong component9() { - return getControlProcessingTime(); - } - - @Override - public ULong component10() { - return getIdleTime(); - } - - @Override - public UInteger component11() { - return getNumWorkers(); - } - - @Override - public UInteger value1() { - return getWorkflowId(); - } - - @Override - public UInteger value2() { - return getExecutionId(); - } - - @Override - public String value3() { - return getOperatorId(); - } - - @Override - public Timestamp value4() { - return getTime(); - } - - @Override - public UInteger value5() { - return getInputTupleCnt(); - } - - @Override - public UInteger value6() { - return getOutputTupleCnt(); - } - - @Override - public Byte value7() { - return getStatus(); - } - - @Override - public ULong value8() { - return getDataProcessingTime(); - } - - @Override - public ULong value9() { - return getControlProcessingTime(); - } - - @Override - public ULong value10() { - return getIdleTime(); - } - - @Override - public UInteger value11() { - return getNumWorkers(); - } - - @Override - public WorkflowRuntimeStatisticsRecord value1(UInteger value) { - setWorkflowId(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value2(UInteger value) { - setExecutionId(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value3(String value) { - setOperatorId(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value4(Timestamp value) { - setTime(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value5(UInteger value) { - setInputTupleCnt(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value6(UInteger value) { - setOutputTupleCnt(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value7(Byte value) { - setStatus(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value8(ULong value) { - setDataProcessingTime(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value9(ULong value) { - setControlProcessingTime(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value10(ULong value) { - setIdleTime(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord value11(UInteger value) { - setNumWorkers(value); - return this; - } - - @Override - public WorkflowRuntimeStatisticsRecord values(UInteger value1, UInteger value2, String value3, Timestamp value4, UInteger value5, UInteger value6, Byte value7, ULong value8, ULong value9, ULong value10, UInteger value11) { - value1(value1); - value2(value2); - value3(value3); - value4(value4); - value5(value5); - value6(value6); - value7(value7); - value8(value8); - value9(value9); - value10(value10); - value11(value11); - return this; - } - - // ------------------------------------------------------------------------- - // FROM and INTO - // ------------------------------------------------------------------------- - - @Override - public void from(IWorkflowRuntimeStatistics from) { - setWorkflowId(from.getWorkflowId()); - setExecutionId(from.getExecutionId()); - setOperatorId(from.getOperatorId()); - setTime(from.getTime()); - setInputTupleCnt(from.getInputTupleCnt()); - setOutputTupleCnt(from.getOutputTupleCnt()); - setStatus(from.getStatus()); - setDataProcessingTime(from.getDataProcessingTime()); - setControlProcessingTime(from.getControlProcessingTime()); - setIdleTime(from.getIdleTime()); - setNumWorkers(from.getNumWorkers()); - } - - @Override - public E into(E into) { - into.from(this); - return into; - } - - // ------------------------------------------------------------------------- - // Constructors - // ------------------------------------------------------------------------- - - /** - * Create a detached WorkflowRuntimeStatisticsRecord - */ - public WorkflowRuntimeStatisticsRecord() { - super(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS); - } - - /** - * Create a detached, initialised WorkflowRuntimeStatisticsRecord - */ - public WorkflowRuntimeStatisticsRecord(UInteger workflowId, UInteger executionId, String operatorId, Timestamp time, UInteger inputTupleCnt, UInteger outputTupleCnt, Byte status, ULong dataProcessingTime, ULong controlProcessingTime, ULong idleTime, UInteger numWorkers) { - super(WorkflowRuntimeStatistics.WORKFLOW_RUNTIME_STATISTICS); - - set(0, workflowId); - set(1, executionId); - set(2, operatorId); - set(3, time); - set(4, inputTupleCnt); - set(5, outputTupleCnt); - set(6, status); - set(7, dataProcessingTime); - set(8, controlProcessingTime); - set(9, idleTime); - set(10, numWorkers); - } -} diff --git a/core/gui/src/app/dashboard/component/user/user-workflow/ngbd-modal-workflow-executions/workflow-runtime-statistics/workflow-runtime-statistics.component.ts b/core/gui/src/app/dashboard/component/user/user-workflow/ngbd-modal-workflow-executions/workflow-runtime-statistics/workflow-runtime-statistics.component.ts index c5cc70d8fba..9bf888166a3 100644 --- a/core/gui/src/app/dashboard/component/user/user-workflow/ngbd-modal-workflow-executions/workflow-runtime-statistics/workflow-runtime-statistics.component.ts +++ b/core/gui/src/app/dashboard/component/user/user-workflow/ngbd-modal-workflow-executions/workflow-runtime-statistics/workflow-runtime-statistics.component.ts @@ -1,6 +1,6 @@ import { Component, inject, OnInit } from "@angular/core"; import { UntilDestroy } from "@ngneat/until-destroy"; -import { OperatorRuntimeStatistics } from "../../../../../type/operator-runtime-statistics"; +import { WorkflowRuntimeStatistics } from "../../../../../type/workflow-runtime-statistics"; import * as Plotly from "plotly.js-basic-dist-min"; import { NzTabChangeEvent } from "ng-zorro-antd/tabs"; import { NZ_MODAL_DATA } from "ng-zorro-antd/modal"; @@ -12,9 +12,9 @@ import { NZ_MODAL_DATA } from "ng-zorro-antd/modal"; styleUrls: ["./workflow-runtime-statistics.component.scss"], }) export class WorkflowRuntimeStatisticsComponent implements OnInit { - readonly workflowRuntimeStatistics: OperatorRuntimeStatistics[] = inject(NZ_MODAL_DATA).workflowRuntimeStatistics; + readonly workflowRuntimeStatistics: WorkflowRuntimeStatistics[] = inject(NZ_MODAL_DATA).workflowRuntimeStatistics; - private groupedStats?: Record; + private groupedStats?: Record; public metrics: string[] = [ "Input Tuple Count", "Output Tuple Count", @@ -43,13 +43,13 @@ export class WorkflowRuntimeStatisticsComponent implements OnInit { /** * Convert an array into a record by combining stats to the same metric and accumulate tuple counts */ - private groupStatsByOperatorId(): Record { + private groupStatsByOperatorId(): Record { if (!this.workflowRuntimeStatistics) { return {}; } const beginTimestamp = this.workflowRuntimeStatistics[0].timestamp; - return this.workflowRuntimeStatistics.reduce((acc: Record, stat) => { + return this.workflowRuntimeStatistics.reduce((acc: Record, stat) => { const statsArray = acc[stat.operatorId] || []; const lastStat = statsArray[statsArray.length - 1]; diff --git a/core/gui/src/app/dashboard/service/user/workflow-executions/workflow-executions.service.ts b/core/gui/src/app/dashboard/service/user/workflow-executions/workflow-executions.service.ts index e2d82d1dff2..fee406b5162 100644 --- a/core/gui/src/app/dashboard/service/user/workflow-executions/workflow-executions.service.ts +++ b/core/gui/src/app/dashboard/service/user/workflow-executions/workflow-executions.service.ts @@ -3,7 +3,7 @@ import { Observable } from "rxjs"; import { AppSettings } from "../../../../common/app-setting"; import { HttpClient } from "@angular/common/http"; import { WorkflowExecutionsEntry } from "../../../type/workflow-executions-entry"; -import { OperatorRuntimeStatistics } from "../../../type/operator-runtime-statistics"; +import { WorkflowRuntimeStatistics } from "../../../type/workflow-runtime-statistics"; export const WORKFLOW_EXECUTIONS_API_BASE_URL = `${AppSettings.getApiEndpoint()}/executions`; @@ -43,7 +43,7 @@ export class WorkflowExecutionsService { }); } - retrieveWorkflowRuntimeStatistics(wid: number, eId: number): Observable { - return this.http.get(`${WORKFLOW_EXECUTIONS_API_BASE_URL}/${wid}/${eId}`); + retrieveWorkflowRuntimeStatistics(wid: number, eId: number): Observable { + return this.http.get(`${WORKFLOW_EXECUTIONS_API_BASE_URL}/${wid}/${eId}`); } } diff --git a/core/gui/src/app/dashboard/type/operator-runtime-statistics.ts b/core/gui/src/app/dashboard/type/workflow-runtime-statistics.ts similarity index 84% rename from core/gui/src/app/dashboard/type/operator-runtime-statistics.ts rename to core/gui/src/app/dashboard/type/workflow-runtime-statistics.ts index cf48d01aa76..d14902438c0 100644 --- a/core/gui/src/app/dashboard/type/operator-runtime-statistics.ts +++ b/core/gui/src/app/dashboard/type/workflow-runtime-statistics.ts @@ -1,4 +1,4 @@ -export interface OperatorRuntimeStatistics { +export interface WorkflowRuntimeStatistics { [key: string]: any; operatorId: string; inputTupleCount: number; diff --git a/core/scripts/sql/texera_ddl.sql b/core/scripts/sql/texera_ddl.sql index 01af5ea1623..244527735ae 100644 --- a/core/scripts/sql/texera_ddl.sql +++ b/core/scripts/sql/texera_ddl.sql @@ -1,7 +1,6 @@ CREATE SCHEMA IF NOT EXISTS `texera_db`; USE `texera_db`; -DROP TABLE IF EXISTS `workflow_runtime_statistics`; DROP TABLE IF EXISTS `workflow_user_access`; DROP TABLE IF EXISTS `workflow_of_user`; DROP TABLE IF EXISTS `user_config`; @@ -14,6 +13,8 @@ DROP TABLE IF EXISTS `workflow_executions`; DROP TABLE IF EXISTS `dataset`; DROP TABLE IF EXISTS `dataset_user_access`; DROP TABLE IF EXISTS `dataset_version`; +DROP TABLE IF EXISTS operator_executions; +DROP TABLE IF EXISTS operator_runtime_statistics; SET PERSIST time_zone = '+00:00'; -- this line is mandatory SET PERSIST sql_mode=(SELECT REPLACE(@@sql_mode,'ONLY_FULL_GROUP_BY','')); @@ -143,24 +144,6 @@ CREATE TABLE IF NOT EXISTS public_project FOREIGN KEY (`pid`) REFERENCES `project` (`pid`) ON DELETE CASCADE ) ENGINE = INNODB; -CREATE TABLE IF NOT EXISTS workflow_runtime_statistics -( - `workflow_id` INT UNSIGNED NOT NULL, - `execution_id` INT UNSIGNED NOT NULL, - `operator_id` VARCHAR(100) NOT NULL, - `time` TIMESTAMP(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6), - `input_tuple_cnt` INT UNSIGNED NOT NULL DEFAULT 0, - `output_tuple_cnt` INT UNSIGNED NOT NULL DEFAULT 0, - `status` TINYINT NOT NULL DEFAULT 1, - `data_processing_time` BIGINT UNSIGNED NOT NULL DEFAULT 0, - `control_processing_time` BIGINT UNSIGNED NOT NULL DEFAULT 0, - `idle_time` BIGINT UNSIGNED NOT NULL DEFAULT 0, - `num_workers` INT UNSIGNED NOT NULL DEFAULT 0, - PRIMARY KEY (`workflow_id`, `execution_id`, `operator_id`, `time`), - FOREIGN KEY (`workflow_id`) REFERENCES `workflow` (`wid`) ON DELETE CASCADE, - FOREIGN KEY (`execution_id`) REFERENCES `workflow_executions` (`eid`) ON DELETE CASCADE -) ENGINE = INNODB; - CREATE TABLE IF NOT EXISTS dataset ( `did` INT UNSIGNED AUTO_INCREMENT NOT NULL, @@ -249,4 +232,26 @@ ALTER TABLE dataset MODIFY COLUMN is_public BOOLEAN NOT NULL DEFAULT true; ALTER TABLE workflow -CHANGE COLUMN is_published is_public BOOLEAN NOT NULL DEFAULT false; \ No newline at end of file +CHANGE COLUMN is_published is_public BOOLEAN NOT NULL DEFAULT false; + +CREATE TABLE IF NOT EXISTS operator_executions ( + operator_execution_id BIGINT UNSIGNED AUTO_INCREMENT PRIMARY KEY, + workflow_execution_id INT UNSIGNED NOT NULL, + operator_id VARCHAR(100) NOT NULL, + UNIQUE (workflow_execution_id, operator_id), + FOREIGN KEY (workflow_execution_id) REFERENCES workflow_executions (eid) ON DELETE CASCADE +); + +CREATE TABLE IF NOT EXISTS operator_runtime_statistics ( + operator_execution_id BIGINT UNSIGNED NOT NULL, + time TIMESTAMP(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6), + input_tuple_cnt BIGINT UNSIGNED NOT NULL DEFAULT 0, + output_tuple_cnt BIGINT UNSIGNED NOT NULL DEFAULT 0, + status TINYINT NOT NULL DEFAULT 1, + data_processing_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + control_processing_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + idle_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + num_workers INT UNSIGNED NOT NULL DEFAULT 0, + PRIMARY KEY (operator_execution_id, time), + FOREIGN KEY (operator_execution_id) REFERENCES operator_executions (operator_execution_id) ON DELETE CASCADE +); diff --git a/core/scripts/sql/update/19.sql b/core/scripts/sql/update/19.sql new file mode 100644 index 00000000000..dafbbbd42d8 --- /dev/null +++ b/core/scripts/sql/update/19.sql @@ -0,0 +1,55 @@ +USE texera_db; +DROP TABLE IF EXISTS operator_runtime_statistics; +DROP TABLE IF EXISTS operator_executions; + +CREATE TABLE IF NOT EXISTS operator_executions ( + operator_execution_id BIGINT UNSIGNED AUTO_INCREMENT PRIMARY KEY, + workflow_execution_id INT UNSIGNED NOT NULL, + operator_id VARCHAR(100) NOT NULL, + UNIQUE (workflow_execution_id, operator_id), + FOREIGN KEY (workflow_execution_id) REFERENCES workflow_executions (eid) ON DELETE CASCADE +); + +CREATE TABLE IF NOT EXISTS operator_runtime_statistics ( + operator_execution_id BIGINT UNSIGNED NOT NULL, + time TIMESTAMP(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6), + input_tuple_cnt BIGINT UNSIGNED NOT NULL DEFAULT 0, + output_tuple_cnt BIGINT UNSIGNED NOT NULL DEFAULT 0, + status TINYINT NOT NULL DEFAULT 1, + data_processing_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + control_processing_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + idle_time BIGINT UNSIGNED NOT NULL DEFAULT 0, + num_workers INT UNSIGNED NOT NULL DEFAULT 0, + PRIMARY KEY (operator_execution_id, time), + FOREIGN KEY (operator_execution_id) REFERENCES operator_executions (operator_execution_id) ON DELETE CASCADE +); + +INSERT IGNORE INTO operator_executions (workflow_execution_id, operator_id) +SELECT DISTINCT execution_id, operator_id +FROM workflow_runtime_statistics; + +INSERT INTO operator_runtime_statistics ( + operator_execution_id, + time, + input_tuple_cnt, + output_tuple_cnt, + status, + data_processing_time, + control_processing_time, + idle_time, + num_workers +) +SELECT + oe.operator_execution_id, + wrs.time, + wrs.input_tuple_cnt, + wrs.output_tuple_cnt, + wrs.status, + wrs.data_processing_time, + wrs.control_processing_time, + wrs.idle_time, + wrs.num_workers +FROM workflow_runtime_statistics wrs +JOIN operator_executions oe ON wrs.execution_id = oe.workflow_execution_id AND wrs.operator_id = oe.operator_id; + +drop table if exists workflow_runtime_statistics;