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

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

                Author: ASF GitHub Bot
            Created on: 22/Sep/22 02:40
            Start Date: 22/Sep/22 02:40
    Worklog Time Spent: 10m 
      Work Description: ZihanLi58 commented on code in PR #3557:
URL: https://github.com/apache/gobblin/pull/3557#discussion_r977082376


##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.monitoring;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.text.StringEscapeUtils;
+
+import com.codahale.metrics.Meter;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.kafka.client.DecodeableKafkaRecord;
+import org.apache.gobblin.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.kafka.HighLevelConsumer;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler;
+
+
+/**
+ * A Flow Spec Store change monitor that uses {@link SpecStoreChangeEvent} 
schema to process Kafka messages received
+ * from the consumer service. This monitor responds to changes to flow specs 
(creations, updates, deletes) and acts as
+ * a connector between the API and execution layers of GaaS.
+ */
+@Slf4j
+public class SpecStoreChangeMonitor extends HighLevelConsumer {
+  public static final String SPEC_STORE_CHANGE_MONITOR_PREFIX = 
"specStoreChangeMonitor";
+  static final String SPEC_STORE_CHANGE_MONITOR_TOPIC_KEY = "topic";
+  static final String SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY = "numThreads";
+
+  // Metrics
+  private Meter successfullyAddedSpecs;
+  private Meter failedAddedSpecs;
+  private Meter deletedSpecs;
+  private Meter unexpectedErrors;
+
+  protected CacheLoader<String, String> cacheLoader = new CacheLoader<String, 
String>() {
+    @Override
+    public String load(String key) throws Exception {
+      return key;
+    }
+  };
+
+  protected LoadingCache<String, String>
+      specChangesSeenCache = CacheBuilder.newBuilder().expireAfterWrite(10, 
TimeUnit.MINUTES).build(cacheLoader);
+
+  @Inject
+  protected FlowCatalog flowCatalog;
+
+  @Inject
+  protected GobblinServiceJobScheduler scheduler;
+
+  public SpecStoreChangeMonitor(String topic, Config config, int numThreads) {
+    super(topic, config, numThreads);
+  }
+
+  @Override
+  /*
+  Note that although this class is multi-threaded and will call this message 
for multiple threads (each having a queue
+  associated with it), a given message itself will be partitioned and assigned 
to only one queue.
+   */
+  protected void processMessage(DecodeableKafkaRecord message) {
+    String specUri = (String) message.getKey();
+    SpecStoreChangeEvent value = (SpecStoreChangeEvent) message.getValue();
+
+    Long timestamp = value.getTimestamp();
+    String operation = value.getOperationType().name();
+    log.info("Processing message with specUri is {} timestamp is {} operation 
is {}", specUri, timestamp, operation);
+
+    // If we've already processed a message with this timestamp and spec uri 
before then skip duplicate message
+    String changeIdentifier = timestamp.toString() + specUri;
+    if (specChangesSeenCache.getIfPresent(changeIdentifier) != null) {
+      return;
+    }
+
+    // If event is a heartbeat type then log it and skip processing
+    if (operation == "HEARTBEAT") {
+      log.debug("Received heartbeat message from time {}", timestamp);
+      return;
+    }
+
+    Spec spec;
+    URI specAsUri = null;
+
+    try {
+      specAsUri = new URI(specUri);
+    } catch (URISyntaxException e) {
+      log.warn("Could not create URI object for specUri {} due to error {}", 
specUri, e.getMessage());
+    }
+
+    spec = (operation != "DELETE") ? 
this.flowCatalog.getSpecWrapper(specAsUri) : null;
+
+    // The monitor should continue to process messages regardless of failures 
with individual messages, instead we use
+    // metrics to keep track of failure to process certain SpecStoreChange 
events
+    try {
+      // Call respective action for the type of change received
+      AddSpecResponse response;
+      if (operation == "INSERT" || operation == "UPDATE") {
+        response = scheduler.onAddSpec(spec);
+
+        // Null response means the dag failed to compile
+        if (response != null && response.getValue() != null) {

Review Comment:
   I feel sometimes we return compile error in the response, you might want to 
catch that scenario as well?
   



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/metrics/RuntimeMetrics.java:
##########
@@ -33,6 +33,10 @@ public class RuntimeMetrics {
   public static final String GOBBLIN_JOB_MONITOR_SLAEVENT_REJECTEDEVENTS = 
"gobblin.jobMonitor.slaevent.rejectedevents";
   public static final String GOBBLIN_JOB_MONITOR_KAFKA_MESSAGE_PARSE_FAILURES =
       "gobblin.jobMonitor.kafka.messageParseFailures";
+  public static final String 
GOBBLIN_SPEC_STORE_MONITOR_SUCCESSFULLY_ADDED_SPECS = 
"gobblin.specStoreMonitor.successful.added.specs";

Review Comment:
   You might want the name to start with 
ServiceMetricNames.GOBBLIN_SERVICE_PREFIX , to make it consistent with other 
metrics and not been filtered out



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitorFactory.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.monitoring;
+
+import java.util.Objects;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+import javax.inject.Provider;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+
+/**
+ * A factory implementation that returns a {@link SpecStoreChangeMonitor} 
instance.
+ */
+@Slf4j
+public class SpecStoreChangeMonitorFactory implements 
Provider<SpecStoreChangeMonitor> {
+  private final Config config;
+
+  @Inject
+  public SpecStoreChangeMonitorFactory(Config config) {
+    this.config = Objects.requireNonNull(config);
+  }
+
+  private SpecStoreChangeMonitor createSpecStoreChangeMonitor()
+      throws ReflectiveOperationException {
+    Config specStoreChangeConfig = 
config.getConfig(SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_PREFIX);
+    String topic = 
specStoreChangeConfig.getString(SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_TOPIC_KEY);
+    int numThreads = ConfigUtils.getInt(specStoreChangeConfig, 
SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY, 5);

Review Comment:
   why not directly define SPEC_STORE_CHANGE_MONITOR_TOPIC_KEY 
    and SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY in this class? 
   Why do we need prefix here, SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY seems 
specifically for spec store change monitor already?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.monitoring;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.text.StringEscapeUtils;
+
+import com.codahale.metrics.Meter;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.kafka.client.DecodeableKafkaRecord;
+import org.apache.gobblin.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.kafka.HighLevelConsumer;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler;
+
+
+/**
+ * A Flow Spec Store change monitor that uses {@link SpecStoreChangeEvent} 
schema to process Kafka messages received
+ * from the consumer service. This monitor responds to changes to flow specs 
(creations, updates, deletes) and acts as
+ * a connector between the API and execution layers of GaaS.
+ */
+@Slf4j
+public class SpecStoreChangeMonitor extends HighLevelConsumer {
+  public static final String SPEC_STORE_CHANGE_MONITOR_PREFIX = 
"specStoreChangeMonitor";
+  static final String SPEC_STORE_CHANGE_MONITOR_TOPIC_KEY = "topic";
+  static final String SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY = "numThreads";
+
+  // Metrics
+  private Meter successfullyAddedSpecs;
+  private Meter failedAddedSpecs;
+  private Meter deletedSpecs;
+  private Meter unexpectedErrors;
+
+  protected CacheLoader<String, String> cacheLoader = new CacheLoader<String, 
String>() {
+    @Override
+    public String load(String key) throws Exception {
+      return key;
+    }
+  };
+
+  protected LoadingCache<String, String>
+      specChangesSeenCache = CacheBuilder.newBuilder().expireAfterWrite(10, 
TimeUnit.MINUTES).build(cacheLoader);
+
+  @Inject
+  protected FlowCatalog flowCatalog;
+
+  @Inject
+  protected GobblinServiceJobScheduler scheduler;
+
+  public SpecStoreChangeMonitor(String topic, Config config, int numThreads) {
+    super(topic, config, numThreads);
+  }
+
+  @Override
+  /*
+  Note that although this class is multi-threaded and will call this message 
for multiple threads (each having a queue
+  associated with it), a given message itself will be partitioned and assigned 
to only one queue.
+   */
+  protected void processMessage(DecodeableKafkaRecord message) {
+    String specUri = (String) message.getKey();
+    SpecStoreChangeEvent value = (SpecStoreChangeEvent) message.getValue();
+
+    Long timestamp = value.getTimestamp();
+    String operation = value.getOperationType().name();
+    log.info("Processing message with specUri is {} timestamp is {} operation 
is {}", specUri, timestamp, operation);
+
+    // If we've already processed a message with this timestamp and spec uri 
before then skip duplicate message
+    String changeIdentifier = timestamp.toString() + specUri;
+    if (specChangesSeenCache.getIfPresent(changeIdentifier) != null) {
+      return;
+    }
+
+    // If event is a heartbeat type then log it and skip processing
+    if (operation == "HEARTBEAT") {
+      log.debug("Received heartbeat message from time {}", timestamp);
+      return;
+    }
+
+    Spec spec;
+    URI specAsUri = null;
+
+    try {
+      specAsUri = new URI(specUri);
+    } catch (URISyntaxException e) {
+      log.warn("Could not create URI object for specUri {} due to error {}", 
specUri, e.getMessage());
+    }
+
+    spec = (operation != "DELETE") ? 
this.flowCatalog.getSpecWrapper(specAsUri) : null;
+
+    // The monitor should continue to process messages regardless of failures 
with individual messages, instead we use
+    // metrics to keep track of failure to process certain SpecStoreChange 
events
+    try {
+      // Call respective action for the type of change received
+      AddSpecResponse response;
+      if (operation == "INSERT" || operation == "UPDATE") {
+        response = scheduler.onAddSpec(spec);
+
+        // Null response means the dag failed to compile
+        if (response != null && response.getValue() != null) {
+          log.info("Successfully added spec {} response {}", spec, 
StringEscapeUtils.escapeJson(response.getValue().toString()));
+          this.successfullyAddedSpecs.mark();
+        } else {
+          log.warn("Failed to add spec {} due to response {}", spec, response);
+          this.failedAddedSpecs.mark();

Review Comment:
   what's the difference between failed added specs and unexpected error?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitorFactory.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.monitoring;
+
+import java.util.Objects;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+import javax.inject.Provider;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+
+/**
+ * A factory implementation that returns a {@link SpecStoreChangeMonitor} 
instance.
+ */
+@Slf4j
+public class SpecStoreChangeMonitorFactory implements 
Provider<SpecStoreChangeMonitor> {
+  private final Config config;
+
+  @Inject
+  public SpecStoreChangeMonitorFactory(Config config) {
+    this.config = Objects.requireNonNull(config);
+  }
+
+  private SpecStoreChangeMonitor createSpecStoreChangeMonitor()
+      throws ReflectiveOperationException {
+    Config specStoreChangeConfig = 
config.getConfig(SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_PREFIX);
+    String topic = 
specStoreChangeConfig.getString(SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_TOPIC_KEY);
+    int numThreads = ConfigUtils.getInt(specStoreChangeConfig, 
SpecStoreChangeMonitor.SPEC_STORE_CHANGE_MONITOR_NUM_THREADS_KEY, 5);
+
+    return (SpecStoreChangeMonitor) 
GobblinConstructorUtils.invokeConstructor(Class.forName("SpecStoreChangeMonitor"),
 topic, specStoreChangeConfig, numThreads);

Review Comment:
   Change SpecStoreChangeMonitor to be a constant value? Also do you need to 
include namespace in the class name as well?





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

    Worklog Id:     (was: 810991)
    Time Spent: 3h 10m  (was: 3h)

> New Consumer service that processes changes to Flow Spec Store
> --------------------------------------------------------------
>
>                 Key: GOBBLIN-1705
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-1705
>             Project: Apache Gobblin
>          Issue Type: New Feature
>          Components: gobblin-service
>            Reporter: Urmi Mustafi
>            Assignee: Abhishek Tiwari
>            Priority: Major
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> In the multi-leader version of GaaS, the REST API layer will not directly 
> contact the `GobblinServiceJobScheduler` to respond to API requests. Instead 
> after flow level updates are persisted to MySQL, this new monitor will 
> subscribe to Kafka events informing it of Flow Spec changes corresponding to 
> the API requests and trigger their execution. There will be a similar change 
> to follow to respond to other API requests. 



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

Reply via email to