Skip to content
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-1979] Pare down TaskStateCollectorService failure logging, to avoid flooding logs during widespread failure, e.g. O(1k)+ #3850

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;

import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.Path;
Expand All @@ -42,6 +43,7 @@
import lombok.extern.slf4j.Slf4j;

import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.exception.RuntimeExceptionWithoutStackTrace;
import org.apache.gobblin.metastore.FsStateStore;
import org.apache.gobblin.metastore.StateStore;
import org.apache.gobblin.runtime.troubleshooter.Issue;
Expand All @@ -50,6 +52,7 @@
import org.apache.gobblin.metrics.event.EventSubmitter;
import org.apache.gobblin.metrics.event.TimingEvent;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;

import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ParallelRunner;
Expand Down Expand Up @@ -252,22 +255,32 @@ public boolean apply(String input) {
}

final Queue<TaskState> taskStateQueue = Queues.newConcurrentLinkedQueue();
AtomicLong numStateStoreMissing = new AtomicLong(0L);
GrowthMilestoneTracker growthTracker = new GrowthMilestoneTracker();
try (ParallelRunner stateSerDeRunner = new ParallelRunner(numDeserializerThreads, null)) {
for (final String taskStateName : taskStateNames) {
log.debug("Found output task state file " + taskStateName);
// Deserialize the TaskState and delete the file
stateSerDeRunner.submitCallable(new Callable<Void>() {
@Override
public Void call() throws Exception {
TaskState taskState = taskStateStore.getAll(taskStateTableName, taskStateName).get(0);
taskStateQueue.add(taskState);
List<TaskState> matchingTaskStates = taskStateStore.getAll(taskStateTableName, taskStateName);
if (matchingTaskStates.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This PR only reduces logs in the case where we are missing task states, but if multiple tasks have the same reason of failure they are still not pared down?

Copy link
Contributor Author

@phet phet Dec 22, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

correct: this solely addresses cases where the state store does not retrieve the task state, but otherwise exits normally. perhaps in another sort of failure, a state store impl might throw. this consolidation still permits such failure to pass through uninterrupted.

since the state store already gave us the list of task state names on line 244, I'd expect any other such failure to be ephemeral (else an abject logical bug in the state store). either way, I've avoided over-engineering the solution, precisely, as you point out, because we'd lose valuable debugging info by conflating dissimilar errors.

if a future failure scenario should arise from which we gain a concrete grasp on what kind of errors these might be, I'd suggest at that time to extend this solution.

long currNumMissing = numStateStoreMissing.incrementAndGet();
// only log selective milestones to avoid flooding log w/ O(100k) stacktraces
if (growthTracker.isAnotherMilestone(currNumMissing)) {
throw new RuntimeExceptionWithoutStackTrace("missing task state [running total: " + currNumMissing + "] - " + taskStateName);
}
return null; // otherwise, when not a milestone, silently skip
}
taskStateQueue.add(matchingTaskStates.get(0));
taskStateStore.delete(taskStateTableName, taskStateName);
return null;
}
}, "Deserialize state for " + taskStateName);
}
} catch (IOException ioe) {
log.error("Could not read all task state files due to", ioe);
log.error("Could not read all task state files [missing final total: " + numStateStoreMissing.get() + "] - ", ioe);
}
log.info(String.format("Collected task state of %d completed tasks in %s", taskStateQueue.size(), taskStateTableName));
return Optional.of(taskStateQueue);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.exception;


/** {@link RuntimeException} that omits the stack trace to streamline both instantiation cost and log volume */
public class RuntimeExceptionWithoutStackTrace extends RuntimeException {
public RuntimeExceptionWithoutStackTrace(String message) {
super(message);
}

public RuntimeExceptionWithoutStackTrace(String message, Throwable cause) {
super(message, cause);
}

/** Secret sauce: no-op */
@Override
public synchronized Throwable fillInStackTrace() {
return this;
}
}
Loading