public Response(Params params) { _status = params.getStatus(); _offset = params.getOffset(); _buildTimeSeconds = params.getBuildTimeSeconds(); _splitCommit = params.isSplitCommit(); _segmentLocation = params.getSegmentLocation(); _controllerVipUrl = params.getControllerVipUrl(); }
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())); }
private SegmentCompletionProtocol.Response keep(String instanceId, long offset) { LOGGER.info("{}:KEEP for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response(new SegmentCompletionProtocol.Response.Params().withOffset(offset) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.KEEP)); }
@Test public void testIncompleteResponseParams() { // Test with reduced params SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS).withOffset(OFFSET) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params); assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS); assertEquals(response.getOffset(), OFFSET); assertNull(response.getSegmentLocation()); assertFalse(response.isSplitCommit()); assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); }
@Test public void testJsonResponseWithoutSplitCommit() { SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS).withOffset(OFFSET) .withSplitCommit(false).withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params); JsonNode jsonNode = JsonUtils.objectToJsonNode(response); assertEquals(jsonNode.get("offset").asInt(), OFFSET); assertNull(jsonNode.get("segmentLocation")); assertFalse(jsonNode.get("isSplitCommitType").asBoolean()); assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString()); assertNull(jsonNode.get("controllerVipUrl")); }
@Test public void testWinnerOnTimeLimit() throws Exception { SegmentCompletionProtocol.Response response; Request.Params params; segmentCompletionMgr._secconds = 10L; params = new Request.Params().withInstanceId(s1).withOffset(s1Offset).withSegmentName(segmentNameStr) .withReason(SegmentCompletionProtocol.REASON_TIME_LIMIT); response = segmentCompletionMgr.segmentConsumed(params); Assert.assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.HOLD); }
private SegmentCompletionProtocol.Response commit(String instanceId, long offset) { long allowedBuildTimeSec = (_maxTimeAllowedToCommitMs - _startTimeMs) / 1000; LOGGER .info("{}:COMMIT for instance={} offset={} buldTimeSec={}", _state, instanceId, offset, allowedBuildTimeSec); SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params().withOffset(offset).withBuildTimeSeconds(allowedBuildTimeSec) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT) .withSplitCommit(_isSplitCommitEnabled); if (_isSplitCommitEnabled) { params.withControllerVipUrl(_controllerVipUrl); } return new SegmentCompletionProtocol.Response(params); }
public static CommittingSegmentDescriptor fromSegmentCompletionReqParams( SegmentCompletionProtocol.Request.Params reqParams) { CommittingSegmentDescriptor committingSegmentDescriptor = new CommittingSegmentDescriptor(reqParams.getSegmentName(), reqParams.getOffset(), reqParams.getSegmentSizeBytes()); committingSegmentDescriptor.setSegmentLocation(reqParams.getSegmentLocation()); return committingSegmentDescriptor; }
public SegmentCompletionProtocol.Response segmentCommit(SegmentCompletionProtocol.Request.Params params, final File segmentTarFile) { SegmentCompletionProtocol.SegmentCommitRequest request = new SegmentCompletionProtocol.SegmentCommitRequest(params); String url = createSegmentCompletionUrl(request); if (url == null) { return SegmentCompletionProtocol.RESP_NOT_SENT; } return uploadSegment(url, params.getSegmentName(), segmentTarFile); }
public SegmentCompletionProtocol.Response segmentCommitStart(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.SegmentCommitStartRequest request = new SegmentCompletionProtocol.SegmentCommitStartRequest(params); String url = createSegmentCompletionUrl(request); if (url == null) { return SegmentCompletionProtocol.RESP_NOT_SENT; } return sendRequest(url); }
public SegmentCompletionProtocol.Response extendBuildTime(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.ExtendBuildTimeRequest request = new SegmentCompletionProtocol.ExtendBuildTimeRequest(params); String url = createSegmentCompletionUrl(request); return sendRequest(url); }
public SegmentCompletionProtocol.Response segmentConsumed(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.SegmentConsumedRequest request = new SegmentCompletionProtocol.SegmentConsumedRequest(params); String url = createSegmentCompletionUrl(request); if (url == null) { return SegmentCompletionProtocol.RESP_NOT_SENT; } return sendRequest(url); }
public SegmentCompletionProtocol.Response segmentCommitEnd(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.SegmentCommitEndRequest request = new SegmentCompletionProtocol.SegmentCommitEndRequest(params); String url = createSegmentCompletionUrl(request); if (url == null) { return SegmentCompletionProtocol.RESP_NOT_SENT; } return sendRequest(url); }
public SegmentCompletionProtocol.Response segmentStoppedConsuming(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.SegmentStoppedConsuming request = new SegmentCompletionProtocol.SegmentStoppedConsuming(params); String url = createSegmentCompletionUrl(request); if (url == null) { return SegmentCompletionProtocol.RESP_NOT_SENT; } return sendRequest(url); }
@Test public void testCompleteResponseParams() { // Test with all params SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS).withOffset(OFFSET) .withSegmentLocation(SEGMENT_LOCATION).withSplitCommit(true) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params); assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS); assertEquals(response.getOffset(), OFFSET); assertEquals(response.getSegmentLocation(), SEGMENT_LOCATION); assertTrue(response.isSplitCommit()); assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); }
@Test public void testJsonNullSegmentLocationAndVip() { SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS).withOffset(OFFSET) .withSplitCommit(false).withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT); SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params); JsonNode jsonNode = JsonUtils.objectToJsonNode(response); assertEquals(jsonNode.get("offset").asInt(), OFFSET); assertNull(jsonNode.get("segmentLocation")); assertFalse(jsonNode.get("isSplitCommitType").asBoolean()); assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString()); assertNull(jsonNode.get("controllerVipUrl")); }
@Test public void testExceptionInConsumedMessage() throws Exception { segmentManager._segmentMetadata = null; SegmentCompletionProtocol.Response response; Request.Params params; segmentCompletionMgr._secconds = 10; params = new Request.Params().withInstanceId(s1).withOffset(s1Offset).withSegmentName(segmentNameStr); response = segmentCompletionMgr.segmentConsumed(params); Assert.assertEquals(response.getStatus(), ControllerResponseStatus.FAILED); }
private SegmentCompletionProtocol.Response hold(String instanceId, long offset) { LOGGER.info("{}:HOLD for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response(new SegmentCompletionProtocol.Response.Params() .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.HOLD).withOffset(offset)); }
private SegmentCompletionProtocol.Response catchup(String instanceId, long offset) { LOGGER.info("{}:CATCHUP for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response( new SegmentCompletionProtocol.Response.Params().withOffset(_winningOffset) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.CATCH_UP)); }
private SegmentCompletionProtocol.Response processCommitWhileUploading(String instanceId, long offset, long now) { LOGGER.info("Processing segmentCommit({}, {})", instanceId, offset); SegmentCompletionProtocol.Response response = abortIfTooLateAndReturnHold(now, instanceId, offset); if (response != null) { return response; } // Another committer (or same) came in while one was uploading. Ask them to hold in case this one fails. return new SegmentCompletionProtocol.Response(new SegmentCompletionProtocol.Response.Params().withOffset(offset) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.HOLD)); }