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

Force commit consuming segments #9197

Merged
merged 2 commits into from
Aug 29, 2022
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 @@ -90,6 +90,26 @@ public Response resumeConsumption(
}
}

@POST
@Path("/tables/{tableName}/forceCommit")
@ApiOperation(value = "Force commit the current consuming segments",
notes = "Force commit the current segments in consuming state and restart consumption. "
+ "This should be used after schema/table config changes. "
+ "Please note that this is an asynchronous operation, "
+ "and 200 response does not mean it has actually been done already")
public Response forceCommit(
@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggest adding an optional segment so that we only force commit one segment?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We don't have partition level pausing. I know this endpoint is a bit different than pause/resume, but let me think about it a bit and maybe we can come up with a way for partition leve operation on all endpoints in this class.

String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(tableName);
validate(tableNameWithType);
try {
_pinotLLCRealtimeSegmentManager.forceCommit(tableNameWithType);
return Response.ok().build();
} catch (Exception e) {
throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR, e);
}
}


@GET
@Path("/tables/{tableName}/pauseStatus")
@Produces(MediaType.APPLICATION_JSON)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1420,6 +1420,15 @@ public void uploadToDeepStoreIfMissing(TableConfig tableConfig, List<SegmentZKMe
}
}

/**
* Force commit the current segments in consuming state and restart consumption
*/
public void forceCommit(String tableNameWithType) {
IdealState idealState = getIdealState(tableNameWithType);
Set<String> consumingSegments = findConsumingSegments(idealState);
sendForceCommitMessageToServers(tableNameWithType, consumingSegments);
}

/**
* Pause consumption on a table by
* 1) setting "isTablePaused" in ideal states to true and
Expand Down