private SegmentCompletionProtocol.Response abortAndReturnFailed() { _state = State.ABORTED; _segmentCompletionManager._controllerMetrics .addMeteredTableValue(_segmentName.getTableName(), ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1); return SegmentCompletionProtocol.RESP_FAILED; }
private SegmentCompletionProtocol.Response COMMITTER_NOTIFIED__commit(String instanceId, long offset, long now) { SegmentCompletionProtocol.Response response = null; response = checkBadCommitRequest(instanceId, offset, now); if (response != null) { return response; } LOGGER.info("{}:Uploading for instance={} offset={}", _state, instanceId, offset); _state = State.COMMITTER_UPLOADING; long commitTimeMs = now - _startTimeMs; if (commitTimeMs > _initialCommitTimeMs) { // We assume that the commit time holds for all partitions. It is possible, though, that one partition // commits at a lower time than another partition, and the two partitions are going simultaneously, // and we may not get the maximum value all the time. _segmentCompletionManager._commitTimeMap.put(_segmentName.getTableName(), commitTimeMs); } return SegmentCompletionProtocol.RESP_COMMIT_CONTINUE; }
private SegmentCompletionProtocol.Response abortAndReturnHold(long now, String instanceId, long offset) { _state = State.ABORTED; _segmentCompletionManager._controllerMetrics .addMeteredTableValue(_segmentName.getTableName(), ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1); return hold(instanceId, offset); }
private LLCSegmentName makeNextLLCSegmentName(LLCSegmentName segmentName, int partition, long now) { final int newSeqNum = segmentName.getSequenceNumber() + 1; LLCSegmentName newLLCSegmentName = new LLCSegmentName(segmentName.getTableName(), partition, newSeqNum, now); return newLLCSegmentName; }
private SegmentCompletionFSM(PinotLLCRealtimeSegmentManager segmentManager, SegmentCompletionManager segmentCompletionManager, LLCSegmentName segmentName, int numReplicas) { _segmentName = segmentName; _numReplicas = numReplicas; _segmentManager = segmentManager; _commitStateMap = new HashMap<>(_numReplicas); _excludedServerStateMap = new HashSet<>(_numReplicas); _segmentCompletionManager = segmentCompletionManager; _startTimeMs = _segmentCompletionManager.getCurrentTimeMs(); _maxTimeToPickWinnerMs = _startTimeMs + MAX_TIME_TO_PICK_WINNER_MS; _maxTimeToNotifyWinnerMs = _startTimeMs + MAX_TIME_TO_NOTIFY_WINNER_MS; long initialCommitTimeMs = MAX_TIME_TO_NOTIFY_WINNER_MS + _segmentManager.getCommitTimeoutMS(_segmentName.getTableName()); Long savedCommitTime = _segmentCompletionManager._commitTimeMap.get(segmentName.getTableName()); if (savedCommitTime != null && savedCommitTime > initialCommitTimeMs) { initialCommitTimeMs = savedCommitTime; } LOGGER = LoggerFactory.getLogger("SegmentCompletionFSM_" + segmentName.getSegmentName()); if (initialCommitTimeMs > MAX_COMMIT_TIME_FOR_ALL_SEGMENTS_SECONDS * 1000) { // The table has a really high value configured for max commit time. Set it to a higher value than default // and go from there. LOGGER.info("Configured max commit time {}s too high for table {}, changing to {}s", initialCommitTimeMs / 1000, segmentName.getTableName(), MAX_COMMIT_TIME_FOR_ALL_SEGMENTS_SECONDS); initialCommitTimeMs = MAX_COMMIT_TIME_FOR_ALL_SEGMENTS_SECONDS * 1000; } _initialCommitTimeMs = initialCommitTimeMs; _maxTimeAllowedToCommitMs = _startTimeMs + _initialCommitTimeMs; _isSplitCommitEnabled = segmentCompletionManager.isSplitCommitEnabled(); _controllerVipUrl = segmentCompletionManager.getControllerVipUrl(); }
/** * An instance is reporting that it has stopped consuming a topic due to some error. * Mark the state of the segment to be OFFLINE in idealstate. * When all replicas of this segment are marked offline, the {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}, * in its next run, will auto-create a new segment with the appropriate offset. */ public void segmentStoppedConsuming(final LLCSegmentName segmentName, final String instance) { String rawTableName = segmentName.getTableName(); String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); final String segmentNameStr = segmentName.getSegmentName(); try { HelixHelper.updateIdealState(_helixManager, realtimeTableName, idealState -> { idealState.setPartitionState(segmentNameStr, instance, CommonConstants.Helix.StateModel.SegmentOnlineOfflineStateModel.OFFLINE); Map<String, String> instanceStateMap = idealState.getInstanceStateMap(segmentNameStr); LOGGER.info("Attempting to mark {} offline. Current map:{}", segmentNameStr, instanceStateMap.toString()); return idealState; }, RetryPolicies.exponentialBackoffRetryPolicy(10, 500L, 1.2f)); } catch (Exception e) { LOGGER.error("Failed to update idealstate for table {} instance {} segment {}", realtimeTableName, instance, segmentNameStr, e); _controllerMetrics.addMeteredGlobalValue(ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1); throw e; } LOGGER.info("Successfully marked {} offline for instance {} since it stopped consuming", segmentNameStr, instance); }
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; }
protected RealtimeIndexOffHeapMemoryManager(ServerMetrics serverMetrics, String segmentName) { _serverMetrics = serverMetrics; _segmentName = segmentName; if (SegmentName.isLowLevelConsumerSegmentName(segmentName)) { LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); _tableName = llcSegmentName.getTableName(); } else if (SegmentName.isHighLevelConsumerSegmentName(segmentName)) { HLCSegmentName hlcSegmentName = new HLCSegmentName(segmentName); _tableName = hlcSegmentName.getTableName(); } else { // For testing only _tableName = "NoSuchTable"; } }
final String rawTableName = llcSegmentName.getTableName(); final java.net.URI tableDirURI = ControllerConf.getUriFromPath(StringUtil.join("/", provider.getBaseDataDirURI().toString(), rawTableName));
ZKMetadataProvider.getRealtimeSegmentZKMetadata(_propertyStore, segmentName.getTableName(), segmentNameStr); segmentDataManager.goOnlineFromConsuming(metadata); } catch (InterruptedException e) {
@Override public int compareTo(Object o) { LLCSegmentName other = (LLCSegmentName) o; if (!this.getTableName().equals(other.getTableName())) { throw new RuntimeException( "Cannot compare segment names " + this.getSegmentName() + " and " + other.getSegmentName()); } if (this.getPartitionId() > other.getPartitionId()) { return 1; } else if (this.getPartitionId() < other.getPartitionId()) { return -1; } else { if (this.getSequenceNumber() > other.getSequenceNumber()) { return 1; } else if (this.getSequenceNumber() < other.getSequenceNumber()) { return -1; } else { if (!this.getCreationTime().equals(other.getCreationTime())) { // If sequence number is the same, time cannot be different. throw new RuntimeException( "Cannot compare segment names " + this.getSegmentName() + " and " + other.getSegmentName()); } return 0; } } }
try { final String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(segmentName.getTableName()); LLCRealtimeSegmentZKMetadata segmentMetadata = _segmentManager.getRealtimeSegmentZKMetadata(realtimeTableName, segmentName.getSegmentName(), null);
new LLCSegmentName(committingLLCSegmentName.getTableName(), partitionId, newSeqNum, now); String newSegmentNameStr = newLLCSegmentName.getSegmentName();
Assert.assertEquals(segName1.getCreationTime(), creationTime); Assert.assertEquals(segName1.getSequenceNumber(), sequenceNumber); Assert.assertEquals(segName1.getTableName(), tableName); Assert.assertEquals(segName2.getCreationTime(), creationTime); Assert.assertEquals(segName2.getSequenceNumber(), sequenceNumber); Assert.assertEquals(segName2.getTableName(), tableName);
LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); Assert.assertEquals(llcSegmentName.getPartitionId(), partition); Assert.assertEquals(llcSegmentName.getTableName(), TableNameBuilder.extractRawTableName(tableConfig.getTableName())); Assert.assertEquals(metadata.getNumReplicas(), nReplicas);
assertEquals(llcSegment.getTableName(), "myTable");
final String tableName = new LLCSegmentName(segmentNameStr).getTableName(); Assert.assertNull(commitTimeMap.get(tableName)); segmentCompletionMgr._secconds = startTime;
SegmentCompletionProtocol.Response response; Request.Params params; final String tableName = new LLCSegmentName(segmentNameStr).getTableName();