9aman commented on code in PR #14741: URL: https://github.com/apache/pinot/pull/14741#discussion_r1906391302
########## pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java: ########## @@ -0,0 +1,226 @@ +/** + * 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.pinot.controller.helix.core.realtime; + +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.protocols.SegmentCompletionProtocol; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.URIUtils; +import org.apache.pinot.controller.helix.core.realtime.segment.CommittingSegmentDescriptor; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; + + +public class PauselessSegmentCompletionFSM extends BlockingSegmentCompletionFSM { + public PauselessSegmentCompletionFSM(PinotLLCRealtimeSegmentManager segmentManager, + SegmentCompletionManager segmentCompletionManager, LLCSegmentName segmentName, + SegmentZKMetadata segmentMetadata) { + super(segmentManager, segmentCompletionManager, segmentName, segmentMetadata); + if (segmentMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING) { + StreamPartitionMsgOffsetFactory factory = + _segmentCompletionManager.getStreamPartitionMsgOffsetFactory(_segmentName); + StreamPartitionMsgOffset endOffset = factory.create(segmentMetadata.getEndOffset()); + _state = BlockingSegmentCompletionFSMState.COMMITTED; + _winningOffset = endOffset; + _winner = "UNKNOWN"; + } + } + + /* + * A server has sent segmentConsumed() message. The caller will save the segment if we return + * COMMIT_CONTINUE. We need to verify that it is the same server that we notified as the winner + * and the offset is the same as what is coming in with the commit. We can then move to + * COMMITTER_UPLOADING and wait for the segmentCommitEnd() call. + * + * In case of discrepancy we move the state machine to ABORTED state so that this FSM is removed + * from the map, and things start over. In this case, we respond to the server with a 'hold' so + * that they re-transmit their segmentConsumed() message and start over. + */ + @Override + public SegmentCompletionProtocol.Response segmentCommitStart(SegmentCompletionProtocol.Request.Params reqParams) { + String instanceId = reqParams.getInstanceId(); + StreamPartitionMsgOffset offset = _streamPartitionMsgOffsetFactory.create(reqParams.getStreamPartitionMsgOffset()); + long now = _segmentCompletionManager.getCurrentTimeMs(); + if (_excludedServerStateMap.contains(instanceId)) { + _logger.warn("Not accepting commit from {} since it had stoppd consuming", instanceId); + return SegmentCompletionProtocol.RESP_FAILED; + } + synchronized (this) { + _logger.info("Processing segmentCommitStart({}, {})", instanceId, offset); + switch (_state) { + case PARTIAL_CONSUMING: + return partialConsumingCommit(instanceId, offset, now); + + case HOLDING: + return holdingCommit(instanceId, offset, now); + + case COMMITTER_DECIDED: + return committerDecidedCommit(instanceId, offset, now); + + case COMMITTER_NOTIFIED: + SegmentCompletionProtocol.Response response = committerNotifiedCommit(instanceId, offset, now); + try { + if (response == SegmentCompletionProtocol.RESP_COMMIT_CONTINUE) { + CommittingSegmentDescriptor committingSegmentDescriptor = + CommittingSegmentDescriptor.fromSegmentCompletionReqParams(reqParams); + LOGGER.info( + "Starting to commit changes to ZK and ideal state for the segment:{} as the leader has been selected", + _segmentName); + _segmentManager.commitSegmentStartMetadata( + TableNameBuilder.REALTIME.tableNameWithType(_segmentName.getTableName()), + committingSegmentDescriptor); + } + } catch (Exception e) { + // this aims to handle the failures during commitSegmentStartMetadata + // we abort the state machine to allow commit protocol to start from the beginning + // the server would then retry the commit protocol from the start + return abortAndReturnFailed(); Review Comment: The commitStart call of PauselessFSM is similar to commitEnd call of BlockingFSM. In the commitEnd of BlockingFSM we do catch the exception and abort the FSM along with returning a failure. Aborting FSM helps other replicas to start the commit. I have kept the behavior same as before. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
