[
https://issues.apache.org/jira/browse/GOBBLIN-2115?focusedWorklogId=926523&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-926523
]
ASF GitHub Bot logged work on GOBBLIN-2115:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 18/Jul/24 18:10
Start Date: 18/Jul/24 18:10
Worklog Time Spent: 10m
Work Description: umustafi commented on code in PR #3999:
URL: https://github.com/apache/gobblin/pull/3999#discussion_r1683256692
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+/**
+ * An interface for storing and retrieving currently running {@link
Dag.DagNode<JobExecutionPlan>}s.
+ * Callers should use {@link DagStateStore#writeCheckpoint} to store dags.
After that, to update individual
+ * {@link Dag.DagNode}s, {@link DagStateStoreWithDagNodes#updateDagNode}
should be used.
+ * {@link DagStateStore#cleanUp(DagManager.DagId)} should be used to delete
all the {@link Dag.DagNode}s for a {@link Dag}.
+ */
+public interface DagStateStoreWithDagNodes extends DagStateStore {
+ int updateDagNode(DagManager.DagId dagId, Dag.DagNode<JobExecutionPlan>
dagNode) throws IOException;
+
+ Set<Dag.DagNode<JobExecutionPlan>> getDagNodes(DagManager.DagId dagId)
throws IOException;
+
+ Optional<Dag.DagNode<JobExecutionPlan>> getDagNode(DagNodeId dagNodeId)
throws IOException;
Review Comment:
why is this one optional but the set above is not? Is it bc dagNodeId may
not exist or be cleaned up?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java:
##########
@@ -42,6 +41,8 @@
* An interface to provide abstractions for managing {@link Dag} and {@link
org.apache.gobblin.service.modules.flowgraph.Dag.DagNode} states
* and allows add/delete and other functions.
*/
+// todo - this should merge with DagStateStoreWithDagNodes interface and
`addDagNodeState` and `addDagNode` should merge and rename to
Review Comment:
does `DagStateStoreWithDagNodes` not extend `DagManagementStateStore`? If
not then why?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProcUtils.java:
##########
@@ -169,8 +169,8 @@ public static void
cancelDagNode(Dag.DagNode<JobExecutionPlan> dagNodeToCancel,
dagNodeToCancel.getValue().getJobSpec().getUri());
}
DagManagerUtils.getSpecProducer(dagNodeToCancel).cancelJob(dagNodeToCancel.getValue().getJobSpec().getUri(),
props).get();
- // todo - why was it not being cleaned up in DagManager?
- dagManagementStateStore.deleteDagNodeState(dagId, dagNodeToCancel);
+ // add back the dag node with updated states in the store
+ dagManagementStateStore.addDagNodeState(dagNodeToCancel, dagId);
Review Comment:
does this go in failedDagStateStore later once processing the cancellation
event?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+/**
+ * An interface for storing and retrieving currently running {@link
Dag.DagNode<JobExecutionPlan>}s.
+ * Callers should use {@link DagStateStore#writeCheckpoint} to store dags.
After that, to update individual
+ * {@link Dag.DagNode}s, {@link DagStateStoreWithDagNodes#updateDagNode}
should be used.
+ * {@link DagStateStore#cleanUp(DagManager.DagId)} should be used to delete
all the {@link Dag.DagNode}s for a {@link Dag}.
+ */
+public interface DagStateStoreWithDagNodes extends DagStateStore {
+ int updateDagNode(DagManager.DagId dagId, Dag.DagNode<JobExecutionPlan>
dagNode) throws IOException;
Review Comment:
java doc to explain what the return type here is and also add another new
line
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MysqlDagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.lang.reflect.Type;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.eclipse.jgit.errors.NotSupportedException;
+
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonSerializer;
+import com.google.gson.reflect.TypeToken;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.configuration.State;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.metrics.ContextAwareCounter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.runtime.spec_serde.GsonSerDe;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanDagFactory;
+import
org.apache.gobblin.service.modules.spec.JobExecutionPlanListDeserializer;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanListSerializer;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.DBStatementExecutor;
+
+import static
org.apache.gobblin.service.modules.orchestration.DagManagerUtils.generateDagId;
+
+
+/**
+ * An implementation of {@link DagStateStoreWithDagNodes} using MySQL as a
backup.
+ */
+@Slf4j
+public class MysqlDagStateStoreWithDagNodes implements
DagStateStoreWithDagNodes {
+
+ public static final String CONFIG_PREFIX = MysqlDagStateStore.CONFIG_PREFIX;
+ protected final DBStatementExecutor dbStatementExecutor;
+ protected final String tableName;
+ protected final GsonSerDe<List<JobExecutionPlan>> serDe;
+ private final JobExecutionPlanDagFactory jobExecPlanDagFactory;
+
+ // todo add a column that tells if it is a running dag or a failed dag
+ protected static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT
EXISTS %s ("
+ + "dag_node_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_NODE_ID_LENGTH + ")
CHARACTER SET latin1 COLLATE latin1_bin NOT NULL, "
+ + "parent_dag_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_ID_LENGTH + ")
NOT NULL, "
+ + "dag_node JSON NOT NULL, "
+ + "modified_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE
CURRENT_TIMESTAMP, "
+ + "PRIMARY KEY (dag_node_id), "
+ + "UNIQUE INDEX dag_node_index (dag_node_id), "
+ + "INDEX dag_index (parent_dag_id))";
+
+ protected static final String INSERT_STATEMENT = "INSERT INTO %s
(dag_node_id, parent_dag_id, dag_node) "
+ + "VALUES (?, ?, ?) AS new ON DUPLICATE KEY UPDATE dag_node =
new.dag_node";
+ protected static final String GET_DAG_NODES_STATEMENT = "SELECT dag_node
FROM %s WHERE parent_dag_id = ?";
+ protected static final String GET_DAG_NODE_STATEMENT = "SELECT dag_node FROM
%s WHERE dag_node_id = ?";
+ protected static final String DELETE_DAG_STATEMENT = "DELETE FROM %s WHERE
parent_dag_id = ?";
+ private final ContextAwareCounter totalDagCount;
+
+ public MysqlDagStateStoreWithDagNodes(Config config, Map<URI, TopologySpec>
topologySpecMap) throws IOException {
+ if (config.hasPath(CONFIG_PREFIX)) {
+ config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+ }
+
+ String DEFAULT_TABLE_NAME = "dag_node_state_store";
+ this.tableName = ConfigUtils.getString(config,
ConfigurationKeys.STATE_STORE_DB_TABLE_KEY, DEFAULT_TABLE_NAME);
+ // create table if it does not exist
+ DataSource dataSource = MysqlDataSourceFactory.get(config,
SharedResourcesBrokerFactory.getImplicitBroker());
+
+ try (Connection connection = dataSource.getConnection();
+ PreparedStatement createStatement =
connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+ createStatement.executeUpdate();
+ connection.commit();
+ } catch (SQLException e) {
+ throw new IOException("Failure creation table " + tableName, e);
+ }
+ this.dbStatementExecutor = new DBStatementExecutor(dataSource, log);
+
+ JsonSerializer<List<JobExecutionPlan>> serializer = new
JobExecutionPlanListSerializer();
+ JsonDeserializer<List<JobExecutionPlan>> deserializer = new
JobExecutionPlanListDeserializer(topologySpecMap);
+ Type typeToken = new TypeToken<List<JobExecutionPlan>>() {
+ }.getType();
+ this.serDe = new GsonSerDe<>(typeToken, serializer, deserializer);
+ this.jobExecPlanDagFactory = new JobExecutionPlanDagFactory();
+ MetricContext metricContext =
+ Instrumented.getMetricContext(new
State(ConfigUtils.configToProperties(config)), this.getClass());
+ this.totalDagCount =
metricContext.contextAwareCounter(ServiceMetricNames.DAG_COUNT_MYSQL_DAG_STATE_COUNT);
+ log.info("Instantiated {}", getClass().getSimpleName());
+ }
+
+ @Override
+ public void writeCheckpoint(Dag<JobExecutionPlan> dag)
Review Comment:
I thought we won't use this anymore in favor of updateDag?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MysqlDagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.lang.reflect.Type;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.eclipse.jgit.errors.NotSupportedException;
+
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonSerializer;
+import com.google.gson.reflect.TypeToken;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.configuration.State;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.metrics.ContextAwareCounter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.runtime.spec_serde.GsonSerDe;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanDagFactory;
+import
org.apache.gobblin.service.modules.spec.JobExecutionPlanListDeserializer;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanListSerializer;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.DBStatementExecutor;
+
+import static
org.apache.gobblin.service.modules.orchestration.DagManagerUtils.generateDagId;
+
+
+/**
+ * An implementation of {@link DagStateStoreWithDagNodes} using MySQL as a
backup.
+ */
+@Slf4j
+public class MysqlDagStateStoreWithDagNodes implements
DagStateStoreWithDagNodes {
+
+ public static final String CONFIG_PREFIX = MysqlDagStateStore.CONFIG_PREFIX;
+ protected final DBStatementExecutor dbStatementExecutor;
+ protected final String tableName;
+ protected final GsonSerDe<List<JobExecutionPlan>> serDe;
+ private final JobExecutionPlanDagFactory jobExecPlanDagFactory;
+
+ // todo add a column that tells if it is a running dag or a failed dag
+ protected static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT
EXISTS %s ("
+ + "dag_node_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_NODE_ID_LENGTH + ")
CHARACTER SET latin1 COLLATE latin1_bin NOT NULL, "
+ + "parent_dag_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_ID_LENGTH + ")
NOT NULL, "
+ + "dag_node JSON NOT NULL, "
+ + "modified_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE
CURRENT_TIMESTAMP, "
+ + "PRIMARY KEY (dag_node_id), "
+ + "UNIQUE INDEX dag_node_index (dag_node_id), "
+ + "INDEX dag_index (parent_dag_id))";
+
+ protected static final String INSERT_STATEMENT = "INSERT INTO %s
(dag_node_id, parent_dag_id, dag_node) "
+ + "VALUES (?, ?, ?) AS new ON DUPLICATE KEY UPDATE dag_node =
new.dag_node";
+ protected static final String GET_DAG_NODES_STATEMENT = "SELECT dag_node
FROM %s WHERE parent_dag_id = ?";
+ protected static final String GET_DAG_NODE_STATEMENT = "SELECT dag_node FROM
%s WHERE dag_node_id = ?";
+ protected static final String DELETE_DAG_STATEMENT = "DELETE FROM %s WHERE
parent_dag_id = ?";
+ private final ContextAwareCounter totalDagCount;
+
+ public MysqlDagStateStoreWithDagNodes(Config config, Map<URI, TopologySpec>
topologySpecMap) throws IOException {
+ if (config.hasPath(CONFIG_PREFIX)) {
+ config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+ }
+
+ String DEFAULT_TABLE_NAME = "dag_node_state_store";
Review Comment:
put this above as a class field
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MySqlDagManagementStateStore.java:
##########
@@ -62,13 +60,11 @@
*/
@Slf4j
@Singleton
-public class MostlyMySqlDagManagementStateStore implements
DagManagementStateStore {
Review Comment:
where are the sql statements for this class? to create/update the store
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java:
##########
@@ -84,8 +84,6 @@ protected void act(DagManagementStateStore
dagManagementStateStore, Optional<Dag
// todo - add metrics
} else {
DagProcUtils.submitNextNodes(dagManagementStateStore, dag.get(),
getDagId());
- // Checkpoint the dag state, it should have an updated value of dag nodes
Review Comment:
why do we not update the store here? is it done in conclude or only needs to
be done once in initialize?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -167,9 +160,5 @@ private void onJobFinish(DagManagementStateStore
dagManagementStateStore, Dag.Da
default:
log.warn("It should not reach here. Job status {} is unexpected.",
executionStatus);
}
-
- // Checkpoint the dag state, it should have an updated value of dag fields
- dagManagementStateStore.checkpointDag(dag);
Review Comment:
is this removed bc the dag was not updated in this method?
##########
gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/MySqlDagManagementStateStoreTest.java:
##########
@@ -90,41 +103,40 @@ public void testAddDag() throws Exception {
DagManager.DagId dagId2 = DagManagerUtils.generateDagId(dag2);
DagNodeId dagNodeId =
DagManagerUtils.calcJobId(dagNode.getValue().getJobSpec().getConfig());
- this.dagManagementStateStore.checkpointDag(dag);
- this.dagManagementStateStore.checkpointDag(dag2);
+ this.dagManagementStateStore.addDag(dag);
+ this.dagManagementStateStore.addDag(dag2);
this.dagManagementStateStore.addDagNodeState(dagNode, dagId);
this.dagManagementStateStore.addDagNodeState(dagNode2, dagId);
this.dagManagementStateStore.addDagNodeState(dagNode3, dagId2);
- Assert.assertTrue(this.dagManagementStateStore.containsDag(dagId));
- Assert.assertEquals(dag.toString(),
this.dagManagementStateStore.getDag(dagId).get().toString());
+ Assert.assertTrue(compareLists(dag.getNodes(),
this.dagManagementStateStore.getDag(dagId).get().getNodes()));
Assert.assertEquals(dagNode,
this.dagManagementStateStore.getDagNodeWithJobStatus(dagNodeId).getLeft().get());
Set<Dag.DagNode<JobExecutionPlan>> dagNodes =
this.dagManagementStateStore.getDagNodes(dagId);
Assert.assertEquals(2, dagNodes.size());
Assert.assertTrue(dagNodes.contains(dagNode));
Assert.assertTrue(dagNodes.contains(dagNode2));
- this.dagManagementStateStore.deleteDagNodeState(dagId, dagNode);
-
Assert.assertFalse(this.dagManagementStateStore.getDagNodes(dagId).contains(dagNode));
-
Assert.assertTrue(this.dagManagementStateStore.getDagNodes(dagId).contains(dagNode2));
-
Assert.assertTrue(this.dagManagementStateStore.getDagNodes(dagId2).contains(dagNode3));
-
// test to verify that adding a new dag node with the same dag node id
(defined by the jobSpec) replaces the existing one
-
Assert.assertEquals(this.dagManagementStateStore.getDagNodes(dagId).size(), 1);
Review Comment:
do we not want to assert the size before adding the duplicate?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MysqlDagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.lang.reflect.Type;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.eclipse.jgit.errors.NotSupportedException;
+
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonSerializer;
+import com.google.gson.reflect.TypeToken;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.configuration.State;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.metrics.ContextAwareCounter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.runtime.spec_serde.GsonSerDe;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanDagFactory;
+import
org.apache.gobblin.service.modules.spec.JobExecutionPlanListDeserializer;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlanListSerializer;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.DBStatementExecutor;
+
+import static
org.apache.gobblin.service.modules.orchestration.DagManagerUtils.generateDagId;
+
+
+/**
+ * An implementation of {@link DagStateStoreWithDagNodes} using MySQL as a
backup.
+ */
+@Slf4j
+public class MysqlDagStateStoreWithDagNodes implements
DagStateStoreWithDagNodes {
+
+ public static final String CONFIG_PREFIX = MysqlDagStateStore.CONFIG_PREFIX;
+ protected final DBStatementExecutor dbStatementExecutor;
+ protected final String tableName;
+ protected final GsonSerDe<List<JobExecutionPlan>> serDe;
+ private final JobExecutionPlanDagFactory jobExecPlanDagFactory;
+
+ // todo add a column that tells if it is a running dag or a failed dag
+ protected static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT
EXISTS %s ("
+ + "dag_node_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_NODE_ID_LENGTH + ")
CHARACTER SET latin1 COLLATE latin1_bin NOT NULL, "
+ + "parent_dag_id VARCHAR(" + ServiceConfigKeys.MAX_DAG_ID_LENGTH + ")
NOT NULL, "
+ + "dag_node JSON NOT NULL, "
+ + "modified_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE
CURRENT_TIMESTAMP, "
+ + "PRIMARY KEY (dag_node_id), "
+ + "UNIQUE INDEX dag_node_index (dag_node_id), "
+ + "INDEX dag_index (parent_dag_id))";
+
+ protected static final String INSERT_STATEMENT = "INSERT INTO %s
(dag_node_id, parent_dag_id, dag_node) "
+ + "VALUES (?, ?, ?) AS new ON DUPLICATE KEY UPDATE dag_node =
new.dag_node";
+ protected static final String GET_DAG_NODES_STATEMENT = "SELECT dag_node
FROM %s WHERE parent_dag_id = ?";
+ protected static final String GET_DAG_NODE_STATEMENT = "SELECT dag_node FROM
%s WHERE dag_node_id = ?";
+ protected static final String DELETE_DAG_STATEMENT = "DELETE FROM %s WHERE
parent_dag_id = ?";
+ private final ContextAwareCounter totalDagCount;
+
+ public MysqlDagStateStoreWithDagNodes(Config config, Map<URI, TopologySpec>
topologySpecMap) throws IOException {
+ if (config.hasPath(CONFIG_PREFIX)) {
+ config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+ }
+
+ String DEFAULT_TABLE_NAME = "dag_node_state_store";
+ this.tableName = ConfigUtils.getString(config,
ConfigurationKeys.STATE_STORE_DB_TABLE_KEY, DEFAULT_TABLE_NAME);
+ // create table if it does not exist
+ DataSource dataSource = MysqlDataSourceFactory.get(config,
SharedResourcesBrokerFactory.getImplicitBroker());
+
+ try (Connection connection = dataSource.getConnection();
+ PreparedStatement createStatement =
connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+ createStatement.executeUpdate();
+ connection.commit();
+ } catch (SQLException e) {
+ throw new IOException("Failure creation table " + tableName, e);
+ }
+ this.dbStatementExecutor = new DBStatementExecutor(dataSource, log);
+
+ JsonSerializer<List<JobExecutionPlan>> serializer = new
JobExecutionPlanListSerializer();
+ JsonDeserializer<List<JobExecutionPlan>> deserializer = new
JobExecutionPlanListDeserializer(topologySpecMap);
+ Type typeToken = new TypeToken<List<JobExecutionPlan>>() {
+ }.getType();
+ this.serDe = new GsonSerDe<>(typeToken, serializer, deserializer);
+ this.jobExecPlanDagFactory = new JobExecutionPlanDagFactory();
+ MetricContext metricContext =
+ Instrumented.getMetricContext(new
State(ConfigUtils.configToProperties(config)), this.getClass());
+ this.totalDagCount =
metricContext.contextAwareCounter(ServiceMetricNames.DAG_COUNT_MYSQL_DAG_STATE_COUNT);
+ log.info("Instantiated {}", getClass().getSimpleName());
+ }
+
+ @Override
+ public void writeCheckpoint(Dag<JobExecutionPlan> dag)
+ throws IOException {
+ DagManager.DagId dagId = DagManagerUtils.generateDagId(dag);
+ boolean newDag = false;
+ for (Dag.DagNode<JobExecutionPlan> dagNode : dag.getNodes()) {
+ if (updateDagNode(dagId, dagNode) == 1) {
+ newDag = true;
+ }
+ }
+ if (newDag) {
+ this.totalDagCount.inc();
+ }
+ }
+
+ @Override
+ public void cleanUp(Dag<JobExecutionPlan> dag) throws IOException {
+ cleanUp(generateDagId(dag));
+ }
+
+ @Override
+ public boolean cleanUp(DagManager.DagId dagId) throws IOException {
+
dbStatementExecutor.withPreparedStatement(String.format(DELETE_DAG_STATEMENT,
tableName), deleteStatement -> {
+ try {
+ deleteStatement.setString(1, dagId.toString());
+ return deleteStatement.executeUpdate() != 0;
+ } catch (SQLException e) {
+ throw new IOException(String.format("Failure deleting dag for %s",
dagId), e);
+ }}, true);
+ this.totalDagCount.dec();
+ return true;
+ }
+
+ @Override
+ public void cleanUp(String dagId) throws IOException {
+ throw new NotSupportedException(getClass().getSimpleName() + " does not
need this legacy API that originated with "
+ + "the DagManager that is replaced by DagProcessingEngine");
+ }
+
+ @Override
+ public List<Dag<JobExecutionPlan>> getDags() throws IOException {
+ throw new NotSupportedException(getClass().getSimpleName() + " does not
need this legacy API that originated with "
+ + "the DagManager that is replaced by DagProcessingEngine"); }
+
+ @Override
+ public Dag<JobExecutionPlan> getDag(DagManager.DagId dagId) throws
IOException {
+ return convertDagNodesIntoDag(getDagNodes(dagId));
+ }
+
+ @Override
+ public Dag<JobExecutionPlan> getDag(String dagId) throws IOException {
+ throw new NotSupportedException(getClass().getSimpleName() + " does not
need this API");
+ }
+
+ @Override
+ public Set<String> getDagIds() throws IOException {
+ throw new NotSupportedException(getClass().getSimpleName() + " does not
need this API");
+ }
+
+ /**
+ * Get the {@link Dag} out of a {@link State} pocket.
+ */
+ private Dag<JobExecutionPlan>
convertDagNodesIntoDag(Set<Dag.DagNode<JobExecutionPlan>> dagNodes) {
+ if (dagNodes.isEmpty()) {
+ return null;
Review Comment:
its better to use an optional here so we know to check for null
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java:
##########
@@ -66,25 +65,11 @@ public interface DagManagementStateStore {
*/
void checkpointDag(Dag<JobExecutionPlan> dag) throws IOException;
- /**
- @return whether `dagId` is currently known due to {@link
DagManagementStateStore#checkpointDag} but not yet
- {@link DagManagementStateStore#deleteDag}
- */
- boolean containsDag(DagManager.DagId dagId) throws IOException;
-
/**
@return the {@link Dag}, if present
*/
Optional<Dag<JobExecutionPlan>> getDag(DagManager.DagId dagId) throws
IOException;
- /**
- * Delete the {@link Dag} from the backing store, typically called upon
completion of execution.
- * @param dag The dag completed/cancelled execution on {@link
org.apache.gobblin.runtime.api.SpecExecutor}.
- */
- default void deleteDag(Dag<JobExecutionPlan> dag) throws IOException {
- deleteDag(DagManagerUtils.generateDagId(dag));
- }
-
Review Comment:
By your logic, then isn't this method necessary in a situation only dagId is
available not the jobExecutionPlan?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MySqlDagManagementStateStore.java:
##########
Review Comment:
I see we want to always fix forward then
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagStateStoreWithDagNodes.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+/**
+ * An interface for storing and retrieving currently running {@link
Dag.DagNode<JobExecutionPlan>}s.
+ * Callers should use {@link DagStateStore#writeCheckpoint} to store dags.
After that, to update individual
+ * {@link Dag.DagNode}s, {@link DagStateStoreWithDagNodes#updateDagNode}
should be used.
+ * {@link DagStateStore#cleanUp(DagManager.DagId)} should be used to delete
all the {@link Dag.DagNode}s for a {@link Dag}.
+ */
+public interface DagStateStoreWithDagNodes extends DagStateStore {
+ int updateDagNode(DagManager.DagId dagId, Dag.DagNode<JobExecutionPlan>
dagNode) throws IOException;
+
+ Set<Dag.DagNode<JobExecutionPlan>> getDagNodes(DagManager.DagId dagId)
throws IOException;
+
+ Optional<Dag.DagNode<JobExecutionPlan>> getDagNode(DagNodeId dagNodeId)
throws IOException;
Review Comment:
java doc would help
##########
gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/MysqlDagStateStoreWithDagNodesTest.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.config.ConfigBuilder;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
+import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+/**
+ * Mainly testing functionalities related to DagStateStore but not
Mysql-related components.
+ */
+public class MysqlDagStateStoreWithDagNodesTest {
+
+ private DagStateStore dagStateStore;
+
+ private static final String TEST_USER = "testUser";
+ private static ITestMetastoreDatabase testDb;
+
+ @BeforeClass
+ public void setUp() throws Exception {
+ testDb = TestMetastoreDatabaseFactory.get();
+ ConfigBuilder configBuilder = ConfigBuilder.create()
+ .addPrimitive(ConfigurationKeys.STATE_STORE_DB_USER_KEY, TEST_USER)
+ .addPrimitive(ConfigurationKeys.STATE_STORE_DB_URL_KEY,
testDb.getJdbcUrl())
+ .addPrimitive(ConfigurationKeys.STATE_STORE_DB_PASSWORD_KEY,
MySqlDagManagementStateStoreTest.TEST_PASSWORD);
+
+ // Constructing TopologySpecMap.
+ Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ String specExecInstance = "mySpecExecutor";
+ TopologySpec topologySpec =
DagTestUtils.buildNaiveTopologySpec(specExecInstance);
+ URI specExecURI = new URI(specExecInstance);
+ topologySpecMap.put(specExecURI, topologySpec);
+ this.dagStateStore = new
MysqlDagStateStoreWithDagNodes(configBuilder.build(), topologySpecMap);
+ }
+
+ @AfterClass(alwaysRun = true)
+ public void tearDown() throws Exception {
+ if (testDb != null) {
+ // `.close()` to avoid (in the aggregate, across multiple suites) -
java.sql.SQLNonTransientConnectionException: Too many connections
+ testDb.close();
+ }
+ }
+
+ @Test
+ public void testAddGetAndDeleteDag() throws Exception{
+ Dag<JobExecutionPlan> dag0Orig = DagTestUtils.buildDag("random_0", 123L);
Review Comment:
make the name and id var so easy to assert later
Issue Time Tracking
-------------------
Worklog Id: (was: 926523)
Remaining Estimate: 0h
Time Spent: 10m
> implement fully mysql based DMSS
> --------------------------------
>
> Key: GOBBLIN-2115
> URL: https://issues.apache.org/jira/browse/GOBBLIN-2115
> Project: Apache Gobblin
> Issue Type: Improvement
> Reporter: Arjun Singh Bora
> Priority: Major
> Time Spent: 10m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)