[ 
https://issues.apache.org/jira/browse/GOBBLIN-2115?focusedWorklogId=926538&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-926538
 ]

ASF GitHub Bot logged work on GOBBLIN-2115:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Jul/24 19:36
            Start Date: 18/Jul/24 19:36
    Worklog Time Spent: 10m 
      Work Description: arjun4084346 commented on code in PR #3999:
URL: https://github.com/apache/gobblin/pull/3999#discussion_r1683382853


##########
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:
   as per kip's comments, it is too late to change api signature for 
`DagStateStore.getDag` method, which is the only caller for this...





Issue Time Tracking
-------------------

    Worklog Id:     (was: 926538)
    Time Spent: 1h 10m  (was: 1h)

> 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: 1h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to