-
Notifications
You must be signed in to change notification settings - Fork 751
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[GOBBLIN-1656] Return a http status 503 on GaaS when quota is exceeded for user or flowgroup #3516
Changes from all commits
a217af0
b64ea2e
503b654
14cb970
bf11a57
268bcaa
28f1f4f
7b8cd35
4b99a47
59838f0
1aaf86a
d7e92ea
9b421e5
4b80123
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,6 +42,7 @@ | |
|
||
import org.apache.gobblin.config.ConfigBuilder; | ||
import org.apache.gobblin.configuration.ConfigurationKeys; | ||
import org.apache.gobblin.exception.QuotaExceededException; | ||
import org.apache.gobblin.instrumented.Instrumented; | ||
import org.apache.gobblin.metrics.ContextAwareMeter; | ||
import org.apache.gobblin.metrics.MetricContext; | ||
|
@@ -133,7 +134,15 @@ public CreateResponse createFlowConfig(FlowConfig flowConfig, boolean triggerLis | |
if (!flowConfig.hasSchedule() && this.flowCatalog.exists(flowSpec.getUri())) { | ||
return new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), HttpStatus.S_409_CONFLICT); | ||
} else { | ||
this.flowCatalog.put(flowSpec, triggerListener); | ||
try { | ||
this.flowCatalog.put(flowSpec, triggerListener); | ||
} catch (QuotaExceededException e) { | ||
throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); | ||
} catch (Throwable e) { | ||
// TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings | ||
log.warn(String.format("Failed to add flow configuration %s.%s to catalog due to", flowConfig.getId().getFlowGroup(), flowConfig.getId().getFlowName()), e); | ||
throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage()); | ||
} | ||
return new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), HttpStatus.S_201_CREATED); | ||
} | ||
} | ||
|
@@ -168,8 +177,15 @@ public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, boo | |
originalFlowConfig.setSchedule(NEVER_RUN_CRON_SCHEDULE); | ||
flowConfig = originalFlowConfig; | ||
} | ||
|
||
this.flowCatalog.put(createFlowSpecForConfig(flowConfig), triggerListener); | ||
try { | ||
this.flowCatalog.put(createFlowSpecForConfig(flowConfig), triggerListener); | ||
} catch (QuotaExceededException e) { | ||
throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); | ||
} catch (Throwable e) { | ||
// TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same here There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. likely not major, but I'm unclear specifically what this TODO is suggesting ought to change in the future. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can make a ticket for it, but right now the flow compilation process is really descriptive what is missing, and users would see a generic HTTP status 400 error with a "Path does not exist" due to failing compilation. I think if we utilized exceptions rather than that compilation check it would be easier to differentiate and reason about. |
||
log.warn(String.format("Failed to add flow configuration %s.%sto catalog due to", flowId.getFlowGroup(), flowId.getFlowName()), e); | ||
throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage()); | ||
} | ||
return new UpdateResponse(HttpStatus.S_200_OK); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -48,7 +48,7 @@ public interface MutableSpecCatalog extends SpecCatalog { | |
* on adding a {@link Spec} to the {@link SpecCatalog}. The key for each entry is the name of the {@link SpecCatalogListener} | ||
* and the value is the result of the the action taken by the listener returned as an instance of {@link AddSpecResponse}. | ||
* */ | ||
Map<String, AddSpecResponse> put(Spec spec); | ||
Map<String, AddSpecResponse> put(Spec spec) throws Throwable; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the only newly added exception it seems could be thrown here is There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's a bit convoluted here the In particular in this segment:
|
||
|
||
/** | ||
* Removes an existing {@link Spec} with the given URI. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -48,7 +48,8 @@ public AddSpecCallback(Spec addedSpec) { | |
_addedSpec = addedSpec; | ||
} | ||
|
||
@Override public AddSpecResponse apply(SpecCatalogListener listener) { | ||
@Override | ||
public AddSpecResponse apply(SpecCatalogListener listener) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. curious what changed to lead you to remove There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah sorry that was a mistake, I was changing a lot of function definitions trying to figure out the errors and this slipped through |
||
return listener.onAddSpec(_addedSpec); | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -342,7 +342,7 @@ public Spec getSpecWrapper(URI uri) { | |
* @param triggerListener True if listeners should be notified. | ||
* @return a map of listeners and their {@link AddSpecResponse}s | ||
*/ | ||
public Map<String, AddSpecResponse> put(Spec spec, boolean triggerListener) { | ||
public Map<String, AddSpecResponse> put(Spec spec, boolean triggerListener) throws Throwable { | ||
Map<String, AddSpecResponse> responseMap = new HashMap<>(); | ||
FlowSpec flowSpec = (FlowSpec) spec; | ||
Preconditions.checkState(state() == State.RUNNING, String.format("%s is not running.", this.getClass().getName())); | ||
|
@@ -355,13 +355,21 @@ public Map<String, AddSpecResponse> put(Spec spec, boolean triggerListener) { | |
|
||
if (triggerListener) { | ||
AddSpecResponse<CallbacksDispatcher.CallbackResults<SpecCatalogListener, AddSpecResponse>> response = this.listeners.onAddSpec(flowSpec); | ||
// If flow fails compilation, the result will have a non-empty string with the error | ||
for (Map.Entry<SpecCatalogListener, CallbackResult<AddSpecResponse>> entry : response.getValue().getSuccesses().entrySet()) { | ||
responseMap.put(entry.getKey().getName(), entry.getValue().getResult()); | ||
} | ||
// If flow fails compilation, the result will have a non-empty string with the error | ||
if (response.getValue().getFailures().size() > 0) { | ||
for (Map.Entry<SpecCatalogListener, CallbackResult<AddSpecResponse>> entry : response.getValue().getFailures().entrySet()) { | ||
throw entry.getValue().getError().getCause(); | ||
} | ||
return responseMap; | ||
} | ||
} | ||
AddSpecResponse<String> schedulerResponse = responseMap.getOrDefault(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS, new AddSpecResponse<>(null)); | ||
Will-Lo marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
if (isCompileSuccessful(responseMap)) { | ||
// Check that the flow configuration is valid and matches to a corresponding edge | ||
if (isCompileSuccessful(schedulerResponse.getValue())) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When it exceed quota, why compile is marked as succeed?It's a little confuse to read the code There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe there's a better terminology here: There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we add comments to explain this? Just to make the code more easy to read There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. a large part of this is the tunneling I describe above, but if you actually do need additional info to support |
||
synchronized (syncObject) { | ||
try { | ||
if (!flowSpec.isExplain()) { | ||
|
@@ -384,19 +392,12 @@ public Map<String, AddSpecResponse> put(Spec spec, boolean triggerListener) { | |
return responseMap; | ||
} | ||
|
||
public static boolean isCompileSuccessful(Map<String, AddSpecResponse> responseMap) { | ||
// If we cannot get the response from the scheduler, assume that the flow failed compilation | ||
AddSpecResponse<String> addSpecResponse = responseMap.getOrDefault( | ||
ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS, new AddSpecResponse<>(null)); | ||
return isCompileSuccessful(addSpecResponse.getValue()); | ||
} | ||
|
||
public static boolean isCompileSuccessful(String dag) { | ||
return dag != null && !dag.contains(ConfigException.class.getSimpleName()); | ||
} | ||
|
||
@Override | ||
public Map<String, AddSpecResponse> put(Spec spec) { | ||
public Map<String, AddSpecResponse> put(Spec spec) throws Throwable { | ||
return put(spec, true); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
|
||
import com.google.common.collect.ImmutableMap; | ||
import com.typesafe.config.ConfigFactory; | ||
import java.io.IOException; | ||
import java.util.HashSet; | ||
import java.util.LinkedList; | ||
import java.util.List; | ||
|
@@ -32,6 +33,7 @@ | |
import com.google.common.collect.Lists; | ||
import com.typesafe.config.Config; | ||
|
||
import java.util.stream.Collectors; | ||
import org.apache.gobblin.configuration.ConfigurationKeys; | ||
import org.apache.gobblin.metrics.event.EventSubmitter; | ||
import org.apache.gobblin.metrics.event.TimingEvent; | ||
|
@@ -41,6 +43,7 @@ | |
import org.apache.gobblin.service.ExecutionStatus; | ||
import org.apache.gobblin.service.FlowId; | ||
import org.apache.gobblin.service.RequesterService; | ||
import org.apache.gobblin.service.ServiceRequester; | ||
import org.apache.gobblin.service.modules.flowgraph.Dag; | ||
import org.apache.gobblin.service.modules.flowgraph.Dag.DagNode; | ||
import org.apache.gobblin.service.modules.orchestration.DagManager.FailureOption; | ||
|
@@ -340,4 +343,18 @@ static void emitFlowEvent(Optional<EventSubmitter> eventSubmitter, Dag<JobExecut | |
eventSubmitter.get().getTimingEvent(flowEvent).stop(flowMetadata); | ||
} | ||
} | ||
|
||
static List<String> getDistinctUniqueRequesters(String serializedRequesters) { | ||
List<String> uniqueRequesters; | ||
try { | ||
uniqueRequesters = RequesterService.deserialize(serializedRequesters) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. no biggie, but you could place the |
||
.stream() | ||
.map(ServiceRequester::getName) | ||
.distinct() | ||
.collect(Collectors.toList()); | ||
return uniqueRequesters; | ||
} catch (IOException e) { | ||
throw new RuntimeException("Could not process requesters due to ", e); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I realize this is just a request handler, so not hugely consequential to catch all exceptions, but please at least just log whatever you find here, so it's not a silent/squelched failure we have a hard time determining to be happening one future day.