1
0

[HUDI-2671] Making error -> warn logs from timeline server with concurrent writers for inconsistent state (#4088)

* Making error -> warn logs from timeline server with concurrent writers for inconsistent state

* Fixing bad request response exception for timeline out of sync

* Addressing feedback. removed write concurrency mode depedency
This commit is contained in:
Sivabalan Narayanan
2021-11-25 14:21:32 -05:00
committed by GitHub
parent 7bb90e8caf
commit f692078d32
3 changed files with 76 additions and 7 deletions

View File

@@ -33,7 +33,6 @@ import org.apache.hudi.common.table.view.FileSystemViewManager;
import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView;
import org.apache.hudi.common.table.view.SyncableFileSystemView;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.timeline.service.handlers.BaseFileHandler;
import org.apache.hudi.timeline.service.handlers.FileSliceHandler;
@@ -42,6 +41,7 @@ import org.apache.hudi.timeline.service.handlers.TimelineHandler;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.javalin.BadRequestResponse;
import io.javalin.Context;
import io.javalin.Handler;
import io.javalin.Javalin;
@@ -507,13 +507,19 @@ public class RequestHandler {
+ " but server has the following timeline "
+ viewManager.getFileSystemView(context.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM))
.getTimeline().getInstants().collect(Collectors.toList());
ValidationUtils.checkArgument(!isLocalViewBehind(context), errMsg);
if (isLocalViewBehind(context)) {
throw new BadRequestResponse(errMsg);
}
long endFinalCheck = System.currentTimeMillis();
finalCheckTimeTaken = endFinalCheck - beginFinalCheck;
}
} catch (RuntimeException re) {
success = false;
LOG.error("Got runtime exception servicing request " + context.queryString(), re);
if (re instanceof BadRequestResponse) {
LOG.warn("Bad request response due to client view behind server view. " + re.getMessage());
} else {
LOG.error("Got runtime exception servicing request " + context.queryString(), re);
}
throw re;
} finally {
long endTs = System.currentTimeMillis();