@POST @Path(SegmentCompletionProtocol.MSG_TYPE_COMMIT) @Consumes(MediaType.MULTIPART_FORM_DATA) @Produces(MediaType.APPLICATION_JSON) public String segmentCommit(@QueryParam(SegmentCompletionProtocol.PARAM_INSTANCE_ID) String instanceId, @QueryParam(SegmentCompletionProtocol.PARAM_SEGMENT_NAME) String segmentName, @QueryParam(SegmentCompletionProtocol.PARAM_OFFSET) long offset, @QueryParam(SegmentCompletionProtocol.PARAM_MEMORY_USED_BYTES) long memoryUsedBytes, @QueryParam(SegmentCompletionProtocol.PARAM_BUILD_TIME_MILLIS) long buildTimeMillis, @QueryParam(SegmentCompletionProtocol.PARAM_WAIT_TIME_MILLIS) long waitTimeMillis, @QueryParam(SegmentCompletionProtocol.PARAM_SEGMENT_SIZE_BYTES) long segmentSizeBytes, @QueryParam(SegmentCompletionProtocol.PARAM_ROW_COUNT) int numRows, FormDataMultiPart multiPart) { SegmentCompletionProtocol.Request.Params requestParams = new SegmentCompletionProtocol.Request.Params(); requestParams.withInstanceId(instanceId).withSegmentName(segmentName).withOffset(offset) .withSegmentSizeBytes(segmentSizeBytes).withBuildTimeMillis(buildTimeMillis).withWaitTimeMillis(waitTimeMillis) .withNumRows(numRows).withMemoryUsedBytes(memoryUsedBytes); LOGGER.info("Processing segmentCommit:{}", requestParams.toString()); final SegmentCompletionManager segmentCompletionManager = SegmentCompletionManager.getInstance(); SegmentCompletionProtocol.Response response = segmentCompletionManager.segmentCommitStart(requestParams); if (response.equals(SegmentCompletionProtocol.RESP_COMMIT_CONTINUE)) { // Get the segment and put it in the right place. boolean success = uploadSegment(multiPart, instanceId, segmentName, false) != null; response = segmentCompletionManager.segmentCommitEnd(requestParams, success, false); } LOGGER.info("Response to segmentCommit: instance={} segment={} status={} offset={}", requestParams.getInstanceId(), requestParams.getSegmentName(), response.getStatus(), response.getOffset()); return response.toJsonString(); }
public SegmentCompletionProtocol.Response segmentCommitEnd(SegmentCompletionProtocol.Request.Params reqParams, boolean success, boolean isSplitCommit) { String instanceId = reqParams.getInstanceId(); long offset = reqParams.getOffset(); synchronized (this) { if (_excludedServerStateMap.contains(instanceId)) { LOGGER.warn("Not accepting commitEnd from {} since it had stoppd consuming", instanceId); return abortAndReturnFailed(); } LOGGER.info("Processing segmentCommit({}, {})", instanceId, offset); if (!_state.equals(State.COMMITTER_UPLOADING) || !instanceId.equals(_winner) || offset != _winningOffset) { // State changed while we were out of sync. return a failed commit. LOGGER.warn("State change during upload: state={} segment={} winner={} winningOffset={}", _state, _segmentName.getSegmentName(), _winner, _winningOffset); return abortAndReturnFailed(); } if (!success) { LOGGER.error("Segment upload failed"); return abortAndReturnFailed(); } SegmentCompletionProtocol.Response response = commitSegment(reqParams, isSplitCommit); if (!response.equals(SegmentCompletionProtocol.RESP_COMMIT_SUCCESS)) { return abortAndReturnFailed(); } else { return response; } } }
public SegmentCompletionProtocol.Response extendBuildTime(final SegmentCompletionProtocol.Request.Params reqParams) { if (!isLeader() || !_helixManager.isConnected()) { _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L); return SegmentCompletionProtocol.RESP_NOT_LEADER; } final String segmentNameStr = reqParams.getSegmentName(); final String instanceId = reqParams.getInstanceId(); final long offset = reqParams.getOffset(); final int extTimeSec = reqParams.getExtraTimeSec(); LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr); SegmentCompletionFSM fsm = null; SegmentCompletionProtocol.Response response = SegmentCompletionProtocol.RESP_FAILED; try { fsm = lookupOrCreateFsm(segmentName, SegmentCompletionProtocol.MSG_TYPE_COMMIT); response = fsm.extendBuildTime(instanceId, offset, extTimeSec); } catch (Exception e) { LOGGER.error("Caught exception in extendBuildTime for segment {}", segmentNameStr, e); } if (fsm != null && fsm.isDone()) { LOGGER.info("Removing FSM (if present):{}", fsm.toString()); _fsmMap.remove(segmentNameStr); } return response; }
final String instanceId = reqParams.getInstanceId(); final long offset = reqParams.getOffset(); LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
private SegmentCompletionProtocol.Response commitSegment(SegmentCompletionProtocol.Request.Params reqParams, boolean isSplitCommit) { boolean success; String instanceId = reqParams.getInstanceId(); long offset = reqParams.getOffset(); if (!_state.equals(State.COMMITTER_UPLOADING)) { // State changed while we were out of sync. return a failed commit. LOGGER.warn("State change during upload: state={} segment={} winner={} winningOffset={}", _state, _segmentName.getSegmentName(), _winner, _winningOffset); return SegmentCompletionProtocol.RESP_FAILED; } LOGGER.info("Committing segment {} at offset {} winner {}", _segmentName.getSegmentName(), offset, instanceId); _state = State.COMMITTING; // In case of splitCommit, the segment is uploaded to a unique file name indicated by segmentLocation, // so we need to move the segment file to its permanent location first before committing the metadata. CommittingSegmentDescriptor committingSegmentDescriptor = CommittingSegmentDescriptor.fromSegmentCompletionReqParams(reqParams); if (isSplitCommit) { if (!_segmentManager.commitSegmentFile(_segmentName.getTableName(), committingSegmentDescriptor)) { return SegmentCompletionProtocol.RESP_FAILED; } } success = _segmentManager.commitSegmentMetadata(_segmentName.getTableName(), committingSegmentDescriptor); if (success) { _state = State.COMMITTED; LOGGER.info("Committed segment {} at offset {} winner {}", _segmentName.getSegmentName(), offset, instanceId); return SegmentCompletionProtocol.RESP_COMMIT_SUCCESS; } return SegmentCompletionProtocol.RESP_FAILED; }
/** * This method is to be called when a server reports that it has stopped consuming a real-time segment. * * @return */ public SegmentCompletionProtocol.Response segmentStoppedConsuming( SegmentCompletionProtocol.Request.Params reqParams) { if (!isLeader() || !_helixManager.isConnected()) { _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L); return SegmentCompletionProtocol.RESP_NOT_LEADER; } final String segmentNameStr = reqParams.getSegmentName(); final String instanceId = reqParams.getInstanceId(); final long offset = reqParams.getOffset(); final String reason = reqParams.getReason(); LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr); SegmentCompletionFSM fsm = null; SegmentCompletionProtocol.Response response = SegmentCompletionProtocol.RESP_FAILED; try { fsm = lookupOrCreateFsm(segmentName, SegmentCompletionProtocol.MSG_TYPE_STOPPED_CONSUMING); response = fsm.stoppedConsuming(instanceId, offset, reason); } catch (Exception e) { LOGGER.error("Caught exception in segmentStoppedConsuming for segment {}", segmentNameStr, e); } if (fsm != null && fsm.isDone()) { LOGGER.info("Removing FSM (if present):{}", fsm.toString()); _fsmMap.remove(segmentNameStr); } return response; }
/** * This method is to be called when a server calls in with the segmentConsumed() API, reporting an offset in the stream * that it currently has (i.e. next offset that it will consume, if it continues to consume). */ public SegmentCompletionProtocol.Response segmentConsumed(SegmentCompletionProtocol.Request.Params reqParams) { if (!isLeader() || !_helixManager.isConnected()) { _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L); return SegmentCompletionProtocol.RESP_NOT_LEADER; } final String segmentNameStr = reqParams.getSegmentName(); final String instanceId = reqParams.getInstanceId(); final String stopReason = reqParams.getReason(); final long offset = reqParams.getOffset(); LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr); SegmentCompletionProtocol.Response response = SegmentCompletionProtocol.RESP_FAILED; SegmentCompletionFSM fsm = null; try { fsm = lookupOrCreateFsm(segmentName, SegmentCompletionProtocol.MSG_TYPE_CONSUMED); response = fsm.segmentConsumed(instanceId, offset, stopReason); } catch (Exception e) { // Return failed response } if (fsm != null && fsm.isDone()) { LOGGER.info("Removing FSM (if present):{}", fsm.toString()); _fsmMap.remove(segmentNameStr); } return response; }
public String getUrl(String hostPort, String protocol) { return protocol + "://" + hostPort + "/" + _msgType + "?" + PARAM_SEGMENT_NAME + "=" + _params.getSegmentName() + "&" + PARAM_OFFSET + "=" + _params.getOffset() + "&" + PARAM_INSTANCE_ID + "=" + _params.getInstanceId() + ( _params.getReason() == null ? "" : ("&" + PARAM_REASON + "=" + _params.getReason())) + ( _params.getBuildTimeMillis() <= 0 ? "" : ("&" + PARAM_BUILD_TIME_MILLIS + "=" + _params.getBuildTimeMillis())) + (_params.getWaitTimeMillis() <= 0 ? "" : ("&" + PARAM_WAIT_TIME_MILLIS + "=" + _params.getWaitTimeMillis())) + (_params.getExtraTimeSec() <= 0 ? "" : ("&" + PARAM_EXTRA_TIME_SEC + "=" + _params.getExtraTimeSec())) + ( _params.getMemoryUsedBytes() <= 0 ? "" : ("&" + PARAM_MEMORY_USED_BYTES + "=" + _params.getMemoryUsedBytes())) + (_params.getSegmentSizeBytes() <= 0 ? "" : ("&" + PARAM_SEGMENT_SIZE_BYTES + "=" + _params.getSegmentSizeBytes())) + (_params.getNumRows() <= 0 ? "" : ("&" + PARAM_ROW_COUNT + "=" + _params.getNumRows())) + (_params.getSegmentLocation() == null ? "" : ("&" + PARAM_SEGMENT_LOCATION + "=" + _params.getSegmentLocation())); }