From 6defce36dda637b2692a3fe97ee58e794bcf76b8 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 19 Dec 2024 19:05:10 +0530 Subject: [PATCH 01/65] Controller side changes to allow pauseless ingestion: 1. Changing FSM 2. Changing the 3 steps performed during the commit protocol to update ZK and Ideal state --- .../utils/PauselessConsumptionUtils.java | 60 ++++++ .../BlockingSegmentCompletionFSM.java | 65 ++++--- .../PauselessSegmentCompletionFSM.java | 179 ++++++++++++++++++ .../PinotLLCRealtimeSegmentManager.java | 116 +++++++++--- .../core/realtime/SegmentCompletionFSM.java | 6 +- .../realtime/SegmentCompletionManager.java | 2 +- .../pinot/spi/utils/CommonConstants.java | 1 + 7 files changed, 368 insertions(+), 61 deletions(-) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java create mode 100644 pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java new file mode 100644 index 000000000000..6ffdec9e76cf --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -0,0 +1,60 @@ +/** + * 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.common.utils; + +import java.util.Optional; +import javax.validation.constraints.NotNull; +import org.apache.pinot.spi.config.table.IndexingConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; + + +public class PauselessConsumptionUtils { + private static final String PAUSELESS_CONSUMPTION_ENABLED = "pauselessConsumptionEnabled"; + + private PauselessConsumptionUtils() { + // Private constructor to prevent instantiation of utility class + } + + /** + * Checks if pauseless consumption is enabled for the given table configuration. + * Returns false if any configuration component is missing or if the flag is not set to true. + * + * @param tableConfig The table configuration to check. Must not be null. + * @return true if pauseless consumption is explicitly enabled, false otherwise + * @throws NullPointerException if tableConfig is null + */ + public static boolean isPauselessEnabled(@NotNull TableConfig tableConfig) { + return checkStreamIngestionConfig(tableConfig) || checkIndexingConfig(tableConfig); + } + + private static boolean checkStreamIngestionConfig(TableConfig tableConfig) { + return Optional.ofNullable(tableConfig.getIngestionConfig()).map(IngestionConfig::getStreamIngestionConfig) + .map(StreamIngestionConfig::getStreamConfigMaps).filter(maps -> !maps.isEmpty()).map(maps -> maps.get(0)) + .map(map -> map.get(PAUSELESS_CONSUMPTION_ENABLED)).map(String::valueOf).map(Boolean::parseBoolean) + .orElse(false); + } + + private static boolean checkIndexingConfig(TableConfig tableConfig) { + return Optional.ofNullable(tableConfig.getIndexingConfig()).map(IndexingConfig::getStreamConfigs) + .map(map -> map.get(PAUSELESS_CONSUMPTION_ENABLED)).map(String::valueOf).map(Boolean::parseBoolean) + .orElse(false); + } +} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java index b119928a461f..ac05d5618e58 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java @@ -88,20 +88,20 @@ public enum BlockingSegmentCompletionFSMState { BlockingSegmentCompletionFSMState _state = BlockingSegmentCompletionFSMState.HOLDING; // Typically start off in HOLDING state. final long _startTimeMs; - private final LLCSegmentName _segmentName; - private final String _rawTableName; - private final String _realtimeTableName; - private final int _numReplicas; - private final Set _excludedServerStateMap; - private final Map _commitStateMap; - private final StreamPartitionMsgOffsetFactory _streamPartitionMsgOffsetFactory; - private StreamPartitionMsgOffset _winningOffset = null; - private String _winner; - private final PinotLLCRealtimeSegmentManager _segmentManager; - private final SegmentCompletionManager _segmentCompletionManager; - private final long _maxTimeToPickWinnerMs; - private final long _maxTimeToNotifyWinnerMs; - private final long _initialCommitTimeMs; + protected final LLCSegmentName _segmentName; + protected final String _rawTableName; + protected final String _realtimeTableName; + protected final int _numReplicas; + protected final Set _excludedServerStateMap; + protected final Map _commitStateMap; + protected final StreamPartitionMsgOffsetFactory _streamPartitionMsgOffsetFactory; + protected StreamPartitionMsgOffset _winningOffset = null; + protected String _winner; + protected final PinotLLCRealtimeSegmentManager _segmentManager; + protected final SegmentCompletionManager _segmentCompletionManager; + protected final long _maxTimeToPickWinnerMs; + protected final long _maxTimeToNotifyWinnerMs; + protected final long _initialCommitTimeMs; // Once the winner is notified, they are expected to commit right away. At this point, it is the segment build // time that we need to consider. // We may need to add some time here to allow for getting the lock? For now 0 @@ -242,7 +242,10 @@ public SegmentCompletionProtocol.Response segmentConsumed(String instanceId, Str * that they re-transmit their segmentConsumed() message and start over. */ @Override - public SegmentCompletionProtocol.Response segmentCommitStart(String instanceId, StreamPartitionMsgOffset offset) { + 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); @@ -376,7 +379,7 @@ public SegmentCompletionProtocol.Response segmentCommitEnd(SegmentCompletionProt } // Helper methods that log the current state and the response sent - private SegmentCompletionProtocol.Response fail(String instanceId, StreamPartitionMsgOffset offset) { + protected SegmentCompletionProtocol.Response fail(String instanceId, StreamPartitionMsgOffset offset) { _logger.info("{}:FAIL for instance={} offset={}", _state, instanceId, offset); return SegmentCompletionProtocol.RESP_FAILED; } @@ -398,28 +401,28 @@ private SegmentCompletionProtocol.Response discard(String instanceId, StreamPart return SegmentCompletionProtocol.RESP_DISCARD; } - private SegmentCompletionProtocol.Response keep(String instanceId, StreamPartitionMsgOffset offset) { + protected SegmentCompletionProtocol.Response keep(String instanceId, StreamPartitionMsgOffset offset) { _logger.info("{}:KEEP for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response( new SegmentCompletionProtocol.Response.Params().withStreamPartitionMsgOffset(offset.toString()) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.KEEP)); } - private SegmentCompletionProtocol.Response catchup(String instanceId, StreamPartitionMsgOffset offset) { + protected SegmentCompletionProtocol.Response catchup(String instanceId, StreamPartitionMsgOffset offset) { _logger.info("{}:CATCHUP for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response( new SegmentCompletionProtocol.Response.Params().withStreamPartitionMsgOffset(_winningOffset.toString()) .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.CATCH_UP)); } - private SegmentCompletionProtocol.Response hold(String instanceId, StreamPartitionMsgOffset offset) { + protected SegmentCompletionProtocol.Response hold(String instanceId, StreamPartitionMsgOffset offset) { _logger.info("{}:HOLD for instance={} offset={}", _state, instanceId, offset); return new SegmentCompletionProtocol.Response(new SegmentCompletionProtocol.Response.Params() .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.HOLD) .withStreamPartitionMsgOffset(offset.toString())); } - private SegmentCompletionProtocol.Response abortAndReturnHold(long now, String instanceId, + protected SegmentCompletionProtocol.Response abortAndReturnHold(long now, String instanceId, StreamPartitionMsgOffset offset) { _state = BlockingSegmentCompletionFSMState.ABORTED; _segmentCompletionManager.getControllerMetrics() @@ -427,14 +430,14 @@ private SegmentCompletionProtocol.Response abortAndReturnHold(long now, String i return hold(instanceId, offset); } - private SegmentCompletionProtocol.Response abortAndReturnFailed() { + protected SegmentCompletionProtocol.Response abortAndReturnFailed() { _state = BlockingSegmentCompletionFSMState.ABORTED; _segmentCompletionManager.getControllerMetrics() .addMeteredTableValue(_rawTableName, ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1); return SegmentCompletionProtocol.RESP_FAILED; } - private SegmentCompletionProtocol.Response abortIfTooLateAndReturnHold(long now, String instanceId, + protected SegmentCompletionProtocol.Response abortIfTooLateAndReturnHold(long now, String instanceId, StreamPartitionMsgOffset offset) { if (now > _maxTimeAllowedToCommitMs) { _logger @@ -464,7 +467,7 @@ private SegmentCompletionProtocol.Response partialConsumingConsumed(String insta * message. As long as the committer is not the one who stopped consuming (which we have already checked before * coming here), we will trust the server that this is a valid commit. */ - private SegmentCompletionProtocol.Response partialConsumingCommit(String instanceId, + protected SegmentCompletionProtocol.Response partialConsumingCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { // Do the same as HOLDING__commit return processCommitWhileHoldingOrPartialConsuming(instanceId, offset, now); @@ -510,7 +513,7 @@ private SegmentCompletionProtocol.Response holdingConsumed(String instanceId, St * This not a good state to receive a commit message, but then it may be that the controller * failed over while in the COMMITTER_NOTIFIED state... */ - private SegmentCompletionProtocol.Response holdingCommit(String instanceId, StreamPartitionMsgOffset offset, + protected SegmentCompletionProtocol.Response holdingCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { return processCommitWhileHoldingOrPartialConsuming(instanceId, offset, now); } @@ -565,7 +568,7 @@ private SegmentCompletionProtocol.Response committerDecidedConsumed(String insta * We have already decided who the committer is, but have not let them know yet. So, we don't expect * a commit() call here. */ - private SegmentCompletionProtocol.Response committerDecidedCommit(String instanceId, + protected SegmentCompletionProtocol.Response committerDecidedCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { return processCommitWhileHoldingOrPartialConsuming(instanceId, offset, now); } @@ -621,7 +624,7 @@ private SegmentCompletionProtocol.Response committerNotifiedConsumed(String inst * We have notified the committer. If we get a consumed message from another server, we can ask them to * catchup (if the offset is lower). If anything else, then we pretty much ask them to hold. */ - private SegmentCompletionProtocol.Response committerNotifiedCommit(String instanceId, + protected SegmentCompletionProtocol.Response committerNotifiedCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { SegmentCompletionProtocol.Response response = null; response = checkBadCommitRequest(instanceId, offset, now); @@ -645,7 +648,7 @@ private SegmentCompletionProtocol.Response committerNotifiedStoppedConsuming(Str return processStoppedConsuming(instanceId, offset, reason, false); } - private SegmentCompletionProtocol.Response committerNotifiedExtendBuildTime(String instanceId, + protected SegmentCompletionProtocol.Response committerNotifiedExtendBuildTime(String instanceId, StreamPartitionMsgOffset offset, int extTimeSec, long now) { SegmentCompletionProtocol.Response response = abortIfTooLateAndReturnHold(now, instanceId, offset); if (response == null) { @@ -667,7 +670,7 @@ private SegmentCompletionProtocol.Response committerUploadingConsumed(String ins return processConsumedAfterCommitStart(instanceId, offset, now); } - private SegmentCompletionProtocol.Response committerUploadingCommit(String instanceId, + protected SegmentCompletionProtocol.Response committerUploadingCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { return processCommitWhileUploading(instanceId, offset, now); } @@ -682,7 +685,7 @@ private SegmentCompletionProtocol.Response committingConsumed(String instanceId, return processConsumedAfterCommitStart(instanceId, offset, now); } - private SegmentCompletionProtocol.Response committingCommit(String instanceId, StreamPartitionMsgOffset offset, + protected SegmentCompletionProtocol.Response committingCommit(String instanceId, StreamPartitionMsgOffset offset, long now) { return processCommitWhileUploading(instanceId, offset, now); } @@ -704,7 +707,7 @@ private SegmentCompletionProtocol.Response committedConsumed(String instanceId, return response; } - private SegmentCompletionProtocol.Response committedCommit(String instanceId, StreamPartitionMsgOffset offset) { + protected SegmentCompletionProtocol.Response committedCommit(String instanceId, StreamPartitionMsgOffset offset) { if (offset.compareTo(_winningOffset) == 0) { return keep(instanceId, offset); } @@ -732,7 +735,7 @@ private SegmentCompletionProtocol.Response processStoppedConsuming(String instan } // A common method when the state is > COMMITTER_NOTIFIED. - private SegmentCompletionProtocol.Response processConsumedAfterCommitStart(String instanceId, + protected SegmentCompletionProtocol.Response processConsumedAfterCommitStart(String instanceId, StreamPartitionMsgOffset offset, long now) { SegmentCompletionProtocol.Response response; // We have already picked a winner, and may or many not have heard from them. diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java new file mode 100644 index 000000000000..7d58a2dd9b92 --- /dev/null +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java @@ -0,0 +1,179 @@ +/** + * 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.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(); + } + return response; + case COMMITTER_UPLOADING: + return committerUploadingCommit(instanceId, offset, now); + + case COMMITTING: + return committingCommit(instanceId, offset, now); + + case COMMITTED: + return committedCommit(instanceId, offset); + + case ABORTED: + return hold(instanceId, offset); + + default: + return fail(instanceId, offset); + } + } + } + + @Override + public SegmentCompletionProtocol.Response extendBuildTime(final String instanceId, + final StreamPartitionMsgOffset offset, final int extTimeSec) { + final long now = _segmentCompletionManager.getCurrentTimeMs(); + synchronized (this) { + _logger.info("Processing extendBuildTime({}, {}, {})", instanceId, offset, extTimeSec); + switch (_state) { + case PARTIAL_CONSUMING: + case HOLDING: + case COMMITTER_DECIDED: + case COMMITTER_NOTIFIED: + return fail(instanceId, offset); + case COMMITTER_UPLOADING: + return committerNotifiedExtendBuildTime(instanceId, offset, extTimeSec, now); + case COMMITTING: + case COMMITTED: + case ABORTED: + default: + return fail(instanceId, offset); + } + } + } + + @Override + // A common method when the state is > COMMITTER_NOTIFIED. + protected SegmentCompletionProtocol.Response processConsumedAfterCommitStart(String instanceId, + StreamPartitionMsgOffset offset, long now) { + SegmentCompletionProtocol.Response response; + // We have already picked a winner, and may or many not have heard from them. + // Common case here is that another server is coming back to us with its offset. We either respond back with + // HOLD or CATCHUP. + // It may be that we never heard from the committer, or the committer is taking too long to commit the segment. + // In that case, we abort the FSM and start afresh (i.e, return HOLD). + // If the winner is coming back again, then we have some more conditions to look at. + response = abortIfTooLateAndReturnHold(now, instanceId, offset); + if (response != null) { + return response; + } + if (instanceId.equals(_winner)) { + // The winner is coming back to report its offset. Take a decision based on the offset reported, and whether we + // already notified them + // Winner is supposedly already in the commit call. Something wrong. + LOGGER.warn( + "{}:Aborting FSM because winner is reporting a segment while it is also committing instance={} offset={} " + + "now={}", _state, instanceId, offset, now); + // Ask them to hold, just in case the committer fails for some reason.. + return abortAndReturnHold(now, instanceId, offset); + } else { + // Common case: A different instance is reporting. + if (offset.compareTo(_winningOffset) == 0) { + // The winner has already updated the segment's ZK metadata for the committing segment. + // Additionally, a new consuming segment has been created for pauseless ingestion. + // Return "keep" to allow the server to build the segment and begin ingestion for the new consuming segment. + response = keep(instanceId, offset); + } else if (offset.compareTo(_winningOffset) < 0) { + response = catchup(instanceId, offset); + } else { + // We have not yet committed, so ask the new responder to hold. They may be the new leader in case the + // committer fails. + response = hold(instanceId, offset); + } + } + return response; + } +} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 4ba7cd2208e6..0ba93457a2ef 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -496,33 +496,27 @@ public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescrip committingSegmentDescriptor.setSegmentLocation(uriToMoveTo); } - /** - * This method is invoked after the realtime segment is uploaded but before a response is sent to the server. - * It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore - * records for new segments, and puts them in idealstate in CONSUMING state. - */ - public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { - Preconditions.checkState(!_isStopping, "Segment manager is stopping"); - - try { - _numCompletingSegments.addAndGet(1); - commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor); - } finally { - _numCompletingSegments.addAndGet(-1); - } + private void commitSegmentStartMetadataInternal(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor) { + commitSegmentInternal(realtimeTableName, committingSegmentDescriptor, true); } private void commitSegmentMetadataInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { - String committingSegmentName = committingSegmentDescriptor.getSegmentName(); - LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); - int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); - LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName); + // Validate segment location only for metadata commit if (StringUtils.isBlank(committingSegmentDescriptor.getSegmentLocation())) { - LOGGER.warn("Committing segment: {} was not uploaded to deep store", committingSegmentName); + LOGGER.warn("Committing segment: {} was not uploaded to deep store", + committingSegmentDescriptor.getSegmentName()); _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); } + commitSegmentInternal(realtimeTableName, committingSegmentDescriptor, false); + } + private void commitSegmentInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor, + boolean isStartMetadata) { + String committingSegmentName = committingSegmentDescriptor.getSegmentName(); + LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); + int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); TableConfig tableConfig = getTableConfig(realtimeTableName); InstancePartitions instancePartitions = getConsumingInstancePartitions(tableConfig); IdealState idealState = getIdealState(realtimeTableName); @@ -534,19 +528,23 @@ private void commitSegmentMetadataInternal(String realtimeTableName, /* * Update zookeeper in 3 steps. * - * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE + * Step 1: Update PROPERTYSTORE to change the old segment metadata status to COMMITTING * Step 2: Update PROPERTYSTORE to create the new segment metadata with status IN_PROGRESS * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state. */ - // Step-1 + // Step-1: Update PROPERTYSTORE + LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName); long startTimeNs1 = System.nanoTime(); SegmentZKMetadata committingSegmentZKMetadata = - updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentDescriptor); - // Refresh the Broker routing to reflect the changes in the segment ZK metadata + isStartMetadata ? updateCommittingSegmentZKMetadataToCOMMITTING(realtimeTableName, committingSegmentDescriptor) + : updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentDescriptor); + + // Refresh the Broker routing _helixResourceManager.sendSegmentRefreshMessage(realtimeTableName, committingSegmentName, false, true); - // Step-2 + // Step-2: Create new segment metadata if needed + LOGGER.info("Creating new segment metadata with status IN_PROGRESS: {}", committingSegmentName); long startTimeNs2 = System.nanoTime(); String newConsumingSegmentName = null; if (!isTablePaused(idealState)) { @@ -559,6 +557,11 @@ private void commitSegmentMetadataInternal(String realtimeTableName, long newSegmentCreationTimeMs = getCurrentTimeMs(); LLCSegmentName newLLCSegment = new LLCSegmentName(rawTableName, committingSegmentPartitionGroupId, committingLLCSegment.getSequenceNumber() + 1, newSegmentCreationTimeMs); + // TODO: This code does not support size-based segment thresholds for tables with pauseless enabled. The + // calculation of row thresholds based on segment size depends on the size of the previously committed + // segment. For tables with pauseless mode enabled, this size is unavailable at this step because the + // segment has not yet been built. + createNewSegmentZKMetadata(tableConfig, streamConfigs.get(0), newLLCSegment, newSegmentCreationTimeMs, committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions, partitionIds.size(), numReplicas); @@ -566,7 +569,9 @@ private void commitSegmentMetadataInternal(String realtimeTableName, } } - // Step-3 + // Step-3: Update IdealState + LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName, + newConsumingSegmentName); long startTimeNs3 = System.nanoTime(); SegmentAssignment segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig, _controllerMetrics); @@ -607,6 +612,65 @@ private void commitSegmentMetadataInternal(String realtimeTableName, } } + /** + * This method is invoked after the realtime segment is ingested but before the response is sent to the server to + * build the segment. + * It updates the propertystore segment metadata from IN_PROGRESS to COMMITTING, and also creates new propertystore + * records for new segments, and puts them in idealstate in CONSUMING state. + */ + public void commitSegmentStartMetadata(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor) { + LOGGER.info("commitSegmentStartMetadata: starting segment commit for table:{}, segment: {}", realtimeTableName, + committingSegmentDescriptor.getSegmentName()); + Preconditions.checkState(!_isStopping, "Segment manager is stopping"); + + try { + _numCompletingSegments.addAndGet(1); + commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor); + } finally { + _numCompletingSegments.addAndGet(-1); + } + } + + /** + * Updates segment ZK metadata for the committing segment. + */ + private SegmentZKMetadata updateCommittingSegmentZKMetadataToCOMMITTING(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor) { + String segmentName = committingSegmentDescriptor.getSegmentName(); + LOGGER.info("Updating segment ZK metadata for committing segment: {}", segmentName); + + Stat stat = new Stat(); + SegmentZKMetadata committingSegmentZKMetadata = getSegmentZKMetadata(realtimeTableName, segmentName, stat); + Preconditions.checkState(committingSegmentZKMetadata.getStatus() == Status.IN_PROGRESS, + "Segment status for segment: %s should be IN_PROGRESS, found: %s", segmentName, + committingSegmentZKMetadata.getStatus()); + + // TODO Issue 5953 remove the long parsing once metadata is set correctly. + committingSegmentZKMetadata.setEndOffset(committingSegmentDescriptor.getNextOffset()); + committingSegmentZKMetadata.setStatus(Status.COMMITTING); + + persistSegmentZKMetadata(realtimeTableName, committingSegmentZKMetadata, stat.getVersion()); + return committingSegmentZKMetadata; + } + + /** + * This method is invoked after the realtime segment is uploaded but before a response is sent to the server. + * It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore + * records for new segments, and puts them in idealstate in CONSUMING state. + */ + public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { + Preconditions.checkState(!_isStopping, "Segment manager is stopping"); + + try { + _numCompletingSegments.addAndGet(1); + commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor); + } finally { + _numCompletingSegments.addAndGet(-1); + } + } + + /** * Updates segment ZK metadata for the committing segment. */ @@ -916,7 +980,7 @@ private Map getLatestSegmentZKMetadataMap(String rea * leader of the table. * * During segment commit, we update zookeeper in 3 steps - * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE + * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE/ COMMITTING * Step 2: Update PROPERTYSTORE to create the new segment metadata with status IN_PROGRESS * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionFSM.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionFSM.java index 516ce4c07d93..c62826cb5fe3 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionFSM.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionFSM.java @@ -80,11 +80,11 @@ SegmentCompletionProtocol.Response segmentConsumed(String instanceId, StreamPart * The FSM verifies whether the server is eligible to commit based on its previous * state and the reported offset, and transitions to a committing state if appropriate. * - * @param instanceId The ID of the server instance attempting to commit. - * @param offset The offset being committed by the server. + * @param reqParams The request parameters containing server instance ID, offset, and other + * segment completion protocol information. * @return A response indicating the next action for the server (e.g., CONTINUE or FAILED). */ - SegmentCompletionProtocol.Response segmentCommitStart(String instanceId, StreamPartitionMsgOffset offset); + SegmentCompletionProtocol.Response segmentCommitStart(SegmentCompletionProtocol.Request.Params reqParams); /** * Handles the event where a server indicates it has stopped consuming. diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java index 5bb3f861d7b0..3dbd20974538 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java @@ -210,7 +210,7 @@ public SegmentCompletionProtocol.Response segmentCommitStart( SegmentCompletionProtocol.Response response = SegmentCompletionProtocol.RESP_FAILED; try { fsm = lookupOrCreateFsm(segmentName, SegmentCompletionProtocol.MSG_TYPE_COMMIT); - response = fsm.segmentCommitStart(instanceId, offset); + response = fsm.segmentCommitStart(reqParams); } catch (Exception e) { LOGGER.error("Caught exception in segmentCommitStart for segment {}", segmentNameStr, e); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 8e27bbccef35..9a8e54b59a79 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -1082,6 +1082,7 @@ public static class Segment { public static class Realtime { public enum Status { IN_PROGRESS, // The segment is still consuming data + COMMITTING, // The segment has been consumed but is yet to be build and uploaded by the server DONE, // The segment has finished consumption and has been committed to the segment store UPLOADED; // The segment is uploaded by an external party From 54ab7b3e2fe204d349b2ac2da0c352b51abeab52 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Fri, 20 Dec 2024 11:06:57 +0530 Subject: [PATCH 02/65] Minor improvements to improve readability --- .../utils/PauselessConsumptionUtils.java | 22 ++++--------------- .../PinotLLCRealtimeSegmentManager.java | 12 ++++------ .../pinot/spi/utils/CommonConstants.java | 3 ++- 3 files changed, 10 insertions(+), 27 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java index 6ffdec9e76cf..58220156c3f4 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -18,12 +18,10 @@ */ package org.apache.pinot.common.utils; -import java.util.Optional; +import java.util.Map; import javax.validation.constraints.NotNull; -import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; public class PauselessConsumptionUtils { @@ -42,19 +40,7 @@ private PauselessConsumptionUtils() { * @throws NullPointerException if tableConfig is null */ public static boolean isPauselessEnabled(@NotNull TableConfig tableConfig) { - return checkStreamIngestionConfig(tableConfig) || checkIndexingConfig(tableConfig); - } - - private static boolean checkStreamIngestionConfig(TableConfig tableConfig) { - return Optional.ofNullable(tableConfig.getIngestionConfig()).map(IngestionConfig::getStreamIngestionConfig) - .map(StreamIngestionConfig::getStreamConfigMaps).filter(maps -> !maps.isEmpty()).map(maps -> maps.get(0)) - .map(map -> map.get(PAUSELESS_CONSUMPTION_ENABLED)).map(String::valueOf).map(Boolean::parseBoolean) - .orElse(false); - } - - private static boolean checkIndexingConfig(TableConfig tableConfig) { - return Optional.ofNullable(tableConfig.getIndexingConfig()).map(IndexingConfig::getStreamConfigs) - .map(map -> map.get(PAUSELESS_CONSUMPTION_ENABLED)).map(String::valueOf).map(Boolean::parseBoolean) - .orElse(false); + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig); + return Boolean.parseBoolean(streamConfigMap.getOrDefault(PAUSELESS_CONSUMPTION_ENABLED, "false")); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 0ba93457a2ef..a4cbfae7a209 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -496,10 +496,6 @@ public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescrip committingSegmentDescriptor.setSegmentLocation(uriToMoveTo); } - private void commitSegmentStartMetadataInternal(String realtimeTableName, - CommittingSegmentDescriptor committingSegmentDescriptor) { - commitSegmentInternal(realtimeTableName, committingSegmentDescriptor, true); - } private void commitSegmentMetadataInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { @@ -509,11 +505,11 @@ private void commitSegmentMetadataInternal(String realtimeTableName, committingSegmentDescriptor.getSegmentName()); _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); } - commitSegmentInternal(realtimeTableName, committingSegmentDescriptor, false); + commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor, false); } - private void commitSegmentInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor, - boolean isStartMetadata) { + private void commitSegmentStartMetadataInternal(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor, boolean isStartMetadata) { String committingSegmentName = committingSegmentDescriptor.getSegmentName(); LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); @@ -626,7 +622,7 @@ public void commitSegmentStartMetadata(String realtimeTableName, try { _numCompletingSegments.addAndGet(1); - commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor); + commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor, true); } finally { _numCompletingSegments.addAndGet(-1); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 9a8e54b59a79..e17b6f565735 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -1082,7 +1082,8 @@ public static class Segment { public static class Realtime { public enum Status { IN_PROGRESS, // The segment is still consuming data - COMMITTING, // The segment has been consumed but is yet to be build and uploaded by the server + COMMITTING, // This state will only be utilised by pauseless ingestion when the segment has been consumed but + // is yet to be build and uploaded by the server. DONE, // The segment has finished consumption and has been committed to the segment store UPLOADED; // The segment is uploaded by an external party From 1e40134b944b19d4a8d265a6552d40b1af2fe27e Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Mon, 23 Dec 2024 17:00:59 +0530 Subject: [PATCH 03/65] Server side changes to enable pauseless consumption. The changes include 1. Changes in the commit protocol to start segment commit before the build 2. Changes in the BaseTableDataManager to ensure that the locally built segment is replaced by a downloaded one only when the CRC is present in the ZK Metadata 3. Changes in the download segment method to allow waited download in case of pauseless consumption --- .../metadata/segment/SegmentZKMetadata.java | 4 +- .../utils/PauselessConsumptionUtils.java | 2 +- .../data/manager/BaseTableDataManager.java | 33 +++++--- .../realtime/PauselessSegmentCommitter.java | 78 +++++++++++++++++++ .../realtime/RealtimeSegmentDataManager.java | 34 ++++++++ .../realtime/RealtimeTableDataManager.java | 64 ++++++++++++++- .../realtime/SegmentCommitterFactory.java | 5 ++ .../realtime/SplitSegmentCommitter.java | 10 +-- 8 files changed, 213 insertions(+), 17 deletions(-) create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/PauselessSegmentCommitter.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java index 69cfe0a63a28..81e0b48f7d35 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java @@ -44,6 +44,8 @@ public class SegmentZKMetadata implements ZKMetadata { private boolean _endTimeMsCached; private long _endTimeMs; + public static final long DEFAULT_CRC_VALUE = -1; + public SegmentZKMetadata(String segmentName) { _znRecord = new ZNRecord(segmentName); _simpleFields = _znRecord.getSimpleFields(); @@ -151,7 +153,7 @@ public long getSizeInBytes() { } public long getCrc() { - return _znRecord.getLongField(Segment.CRC, -1); + return _znRecord.getLongField(Segment.CRC, DEFAULT_CRC_VALUE); } public void setCrc(long crc) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java index 58220156c3f4..e51b2a028053 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -40,7 +40,7 @@ private PauselessConsumptionUtils() { * @throws NullPointerException if tableConfig is null */ public static boolean isPauselessEnabled(@NotNull TableConfig tableConfig) { - Map streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig); + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); return Boolean.parseBoolean(streamConfigMap.getOrDefault(PAUSELESS_CONSUMPTION_ENABLED, "false")); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index e3e17a6f4d2f..3f21d3b7c847 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -639,10 +639,19 @@ public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingCon Lock segmentLock = getSegmentLock(segmentName); segmentLock.lock(); try { - // Download segment from deep store if CRC changes or forced to download; - // otherwise, copy backup directory back to the original index directory. - // And then continue to load the segment from the index directory. - boolean shouldDownload = forceDownload || !hasSameCRC(zkMetadata, localMetadata); + /* + Determines if a segment should be downloaded from deep storage based on: + 1. Forced download flag + 2. CRC value presence and mismatch between ZK metadata and local metadata CRC. + The presence of a CRC in ZK metadata is critical for pauseless tables. It confirms that + the COMMIT_END_METADATA call succeeded and that the segment is available in deep store + or with a peer before discarding the local copy. + otherwise, copy backup directory back to the original index directory. + And then continue to load the segment from the index directory. + */ + boolean shouldDownload = + forceDownload || (zkMetadata.getCrc() != SegmentZKMetadata.DEFAULT_CRC_VALUE && !hasSameCRC(zkMetadata, + localMetadata)); if (shouldDownload) { // Create backup directory to handle failure of segment reloading. createBackup(indexDir); @@ -777,7 +786,7 @@ protected File downloadSegment(SegmentZKMetadata zkMetadata) } } - private File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata) + protected File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata) throws Exception { String segmentName = zkMetadata.getSegmentName(); String downloadUrl = zkMetadata.getDownloadUrl(); @@ -827,7 +836,7 @@ private File downloadSegmentFromDeepStore(SegmentZKMetadata zkMetadata) } } - private File downloadSegmentFromPeers(SegmentZKMetadata zkMetadata) + protected File downloadSegmentFromPeers(SegmentZKMetadata zkMetadata) throws Exception { String segmentName = zkMetadata.getSegmentName(); Preconditions.checkState(_peerDownloadScheme != null, "Peer download is not enabled for table: %s", @@ -987,9 +996,15 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); SegmentMetadataImpl segmentMetadata = (segmentDirectory == null) ? null : segmentDirectory.getSegmentMetadata(); - // If the segment doesn't exist on server or its CRC has changed, then we - // need to fall back to download the segment from deep store to load it. - if (segmentMetadata == null || !hasSameCRC(zkMetadata, segmentMetadata)) { + // If the + // 1. Segment doesn't exist on server or + // 2. CRC value is present and mismatch between ZK metadata and local metadata CRC. + // (The presence of a CRC in ZK metadata is critical for pauseless tables. It confirms that + // the COMMIT_END_METADATA call succeeded and that the segment is available in deep store + // or with a peer before discarding the local copy.) + // then we need to fall back to download the segment from deep store to load it. + if (segmentMetadata == null || (zkMetadata.getCrc() != SegmentZKMetadata.DEFAULT_CRC_VALUE && !hasSameCRC( + zkMetadata, segmentMetadata))) { if (segmentMetadata == null) { _logger.info("Segment: {} does not exist", segmentName); } else if (!hasSameCRC(zkMetadata, segmentMetadata)) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/PauselessSegmentCommitter.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/PauselessSegmentCommitter.java new file mode 100644 index 000000000000..f2c922301099 --- /dev/null +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/PauselessSegmentCommitter.java @@ -0,0 +1,78 @@ +/** + * 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.core.data.manager.realtime; + +import java.io.File; +import javax.annotation.Nullable; +import org.apache.pinot.common.protocols.SegmentCompletionProtocol; +import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; +import org.slf4j.Logger; + + +public class PauselessSegmentCommitter extends SplitSegmentCommitter { + public PauselessSegmentCommitter(Logger segmentLogger, ServerSegmentCompletionProtocolHandler protocolHandler, + SegmentCompletionProtocol.Request.Params params, SegmentUploader segmentUploader, + @Nullable String peerDownloadScheme) { + super(segmentLogger, protocolHandler, params, segmentUploader, peerDownloadScheme); + } + + public PauselessSegmentCommitter(Logger segmentLogger, ServerSegmentCompletionProtocolHandler protocolHandler, + SegmentCompletionProtocol.Request.Params params, SegmentUploader segmentUploader) { + super(segmentLogger, protocolHandler, params, segmentUploader); + } + + /** + * Commits a built segment without executing the segmentCommitStart step. This method assumes that + * segmentCommitStart has already been executed prior to building the segment. + * + * The commit process follows these steps: + * 1. Uploads the segment tar file to the designated storage location + * 2. Updates the parameters with the new segment location + * 3. Executes the segment commit end protocol with associated metadata + * + * @param segmentBuildDescriptor Contains the built segment information including the tar file + * and associated metadata files + * @return A SegmentCompletionProtocol.Response object indicating the commit status: + * - Returns the successful commit response if all steps complete successfully + * - Returns RESP_FAILED if either the upload fails or the commit end protocol fails + * + * @see SegmentCompletionProtocol + * @see RealtimeSegmentDataManager.SegmentBuildDescriptor + */ + @Override + public SegmentCompletionProtocol.Response commit( + RealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor) { + File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + + String segmentLocation = uploadSegment(segmentTarFile, _segmentUploader, _params); + if (segmentLocation == null) { + return SegmentCompletionProtocol.RESP_FAILED; + } + _params.withSegmentLocation(segmentLocation); + + SegmentCompletionProtocol.Response commitEndResponse = + _protocolHandler.segmentCommitEndWithMetadata(_params, segmentBuildDescriptor.getMetadataFiles()); + + if (!commitEndResponse.getStatus().equals(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT_SUCCESS)) { + _segmentLogger.warn("CommitEnd failed with response {}", commitEndResponse.toJsonString()); + return SegmentCompletionProtocol.RESP_FAILED; + } + return commitEndResponse; + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 380b358a84ed..848851aa2707 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -50,6 +50,7 @@ import org.apache.pinot.common.protocols.SegmentCompletionProtocol; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.core.data.manager.realtime.RealtimeConsumptionRateManager.ConsumptionRateLimiter; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; @@ -845,6 +846,23 @@ public void run() { // CONSUMING -> ONLINE state transition. segmentLock.lockInterruptibly(); try { + // For tables with pauseless consumption enabled we want to start the commit protocol that + // 1. Updates the endOffset in the ZK metadata for the committing segment + // 2. Creates ZK metadata for the new consuming segment + // 3. Updates the IdealState for committing and new consuming segment to ONLINE and CONSUMING + // respectively. + // See design doc for the new commit protocol: + // https://docs.google.com/document/d/1d-xttk7sXFIOqfyZvYw5W_KeGS6Ztmi8eYevBcCrT_c + if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { + if (!startSegmentCommit()) { + // If for any reason commit failed, we don't want to be in COMMITTING state when we hold. + // Change the state to HOLDING before looping around. + _state = State.HOLDING; + _segmentLogger.info("Could not commit segment: {}. Retrying after hold", _segmentNameStr); + hold(); + break; + } + } long buildTimeSeconds = response.getBuildTimeSeconds(); buildSegmentForCommit(buildTimeSeconds * 1000L); if (_segmentBuildDescriptor == null) { @@ -907,6 +925,22 @@ public void run() { } } + boolean startSegmentCommit() { + SegmentCompletionProtocol.Request.Params params = new SegmentCompletionProtocol.Request.Params(); + params.withSegmentName(_segmentNameStr).withStreamPartitionMsgOffset(_currentOffset.toString()) + .withNumRows(_numRowsConsumed).withInstanceId(_instanceId).withReason(_stopReason); + if (_isOffHeap) { + params.withMemoryUsedBytes(_memoryManager.getTotalAllocatedBytes()); + } + SegmentCompletionProtocol.Response segmentCommitStartResponse = _protocolHandler.segmentCommitStart(params); + if (!segmentCommitStartResponse.getStatus() + .equals(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT_CONTINUE)) { + _segmentLogger.warn("CommitStart failed with response {}", segmentCommitStartResponse.toJsonString()); + return false; + } + return true; + } + @VisibleForTesting protected StreamPartitionMsgOffset extractOffset(SegmentCompletionProtocol.Response response) { return _streamPartitionMsgOffsetFactory.create(response.getStreamPartitionMsgOffset()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 7cb1a7a5bd93..ddadfc11c68d 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.Lock; import java.util.function.BooleanSupplier; import java.util.function.Supplier; @@ -46,6 +47,7 @@ import org.apache.pinot.common.metrics.ServerGauge; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.SegmentUtils; import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.data.manager.BaseTableDataManager; @@ -119,6 +121,10 @@ public class RealtimeTableDataManager extends BaseTableDataManager { public static final long READY_TO_CONSUME_DATA_CHECK_INTERVAL_MS = TimeUnit.SECONDS.toMillis(5); + public static final long TIMEOUT_MINUTES = 5; + public static final long TIMEOUT_MS = TIMEOUT_MINUTES * 60 * 1000; + public static final long SLEEP_INTERVAL_MS = 30000; // 30 seconds sleep interval + // TODO: Change it to BooleanSupplier private final Supplier _isServerReadyToServeQueries; @@ -461,7 +467,14 @@ protected void doAddOnlineSegment(String segmentName) ((RealtimeSegmentDataManager) segmentDataManager).goOnlineFromConsuming(zkMetadata); onConsumingToOnline(segmentName); } else { - replaceSegmentIfCrcMismatch(segmentDataManager, zkMetadata, indexLoadingConfig); + // For pauseless ingestion, the segment is marked ONLINE before it's built and before the COMMIT_END_METADATA + // call completes. + // The server should replace the segment only after the CRC is set by COMMIT_END_METADATA. + // This ensures the segment's download URL is available before discarding the locally built copy, preventing + // data loss if COMMIT_END_METADATA fails. + if (zkMetadata.getCrc() != SegmentZKMetadata.DEFAULT_CRC_VALUE) { + replaceSegmentIfCrcMismatch(segmentDataManager, zkMetadata, indexLoadingConfig); + } } } } @@ -544,6 +557,55 @@ private void doAddConsumingSegment(String segmentName) _logger.info("Added new CONSUMING segment: {}", segmentName); } + @Override + public File downloadSegment(SegmentZKMetadata zkMetadata) + throws Exception { + if (!PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { + return super.downloadSegment(zkMetadata); + } + + final long startTime = System.currentTimeMillis(); + + while (System.currentTimeMillis() - startTime < TIMEOUT_MS) { + // ZK Metadata may change during segment download process; fetch it on every retry. + zkMetadata = fetchZKMetadata(zkMetadata.getSegmentName()); + + if (zkMetadata.getDownloadUrl() != null) { + // The downloadSegment() will throw an exception in case there are some genuine issues. + // We don't want to retry in those scenarios and will throw an exception + return downloadSegmentFromDeepStore(zkMetadata); + } + + if (_peerDownloadScheme != null) { + _logger.info("Peer download is enabled for the segment: {}", zkMetadata.getSegmentName()); + try { + return downloadSegmentFromPeers(zkMetadata); + } catch (Exception e) { + // TODO :in this case we just retry as some of the other servers might be trying to build the + // segment + _logger.warn("Could not download segment: {} from peer", zkMetadata.getSegmentName(), e); + } + } + + long timeElapsed = System.currentTimeMillis() - startTime; + long timeRemaining = TIMEOUT_MS - timeElapsed; + + if (timeRemaining <= 0) { + break; + } + + _logger.info("Sleeping for 30 seconds as the segment url is missing. Time remaining: {} minutes", + Math.round(timeRemaining / 60000.0)); + + // Sleep for the shorter of our normal interval or remaining time + Thread.sleep(Math.min(SLEEP_INTERVAL_MS, timeRemaining)); + } + +// If we exit the loop without returning, throw an exception + throw new TimeoutException("Failed to download segment after " + TIMEOUT_MINUTES + " minutes of retrying. Segment: " + + zkMetadata.getSegmentName()); + } + /** * Sets the default time value in the schema as the segment creation time if it is invalid. Time column is used to * manage the segments, so its values have to be within the valid range. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentCommitterFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentCommitterFactory.java index 4224019ab0e1..8a637b739508 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentCommitterFactory.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentCommitterFactory.java @@ -21,6 +21,7 @@ import java.net.URISyntaxException; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.protocols.SegmentCompletionProtocol; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; @@ -79,6 +80,10 @@ public SegmentCommitter createSegmentCommitter(SegmentCompletionProtocol.Request _protocolHandler.getAuthProvider(), _tableConfig.getTableName()); } + if (PauselessConsumptionUtils.isPauselessEnabled(_tableConfig)) { + return new PauselessSegmentCommitter(_logger, _protocolHandler, params, segmentUploader, + peerSegmentDownloadScheme); + } return new SplitSegmentCommitter(_logger, _protocolHandler, params, segmentUploader, peerSegmentDownloadScheme); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SplitSegmentCommitter.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SplitSegmentCommitter.java index 1e4ebfe1f856..19aea112486e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SplitSegmentCommitter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SplitSegmentCommitter.java @@ -35,11 +35,11 @@ * If that succeeds, swap in-memory segment with the one built. */ public class SplitSegmentCommitter implements SegmentCommitter { - private final SegmentCompletionProtocol.Request.Params _params; - private final ServerSegmentCompletionProtocolHandler _protocolHandler; - private final SegmentUploader _segmentUploader; - private final String _peerDownloadScheme; - private final Logger _segmentLogger; + protected final SegmentCompletionProtocol.Request.Params _params; + protected final ServerSegmentCompletionProtocolHandler _protocolHandler; + protected final SegmentUploader _segmentUploader; + protected final String _peerDownloadScheme; + protected final Logger _segmentLogger; public SplitSegmentCommitter(Logger segmentLogger, ServerSegmentCompletionProtocolHandler protocolHandler, SegmentCompletionProtocol.Request.Params params, SegmentUploader segmentUploader, From 012da87cb91790add75ab65311ecf97729fd0b69 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Tue, 24 Dec 2024 15:28:04 +0530 Subject: [PATCH 04/65] Fixes in the commit protocol for pauseless ingestion to complete the segment commit end metadata call Refactoing code for redability --- .../utils/PauselessConsumptionUtils.java | 2 +- .../BlockingSegmentCompletionFSM.java | 6 +- .../PauselessSegmentCompletionFSM.java | 47 ++++ .../PinotLLCRealtimeSegmentManager.java | 208 +++++++++++------- ...elessRealtimeIngestionIntegrationTest.java | 166 ++++++++++++++ 5 files changed, 351 insertions(+), 78 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java index e51b2a028053..9aedbbfa180d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -25,7 +25,7 @@ public class PauselessConsumptionUtils { - private static final String PAUSELESS_CONSUMPTION_ENABLED = "pauselessConsumptionEnabled"; + public static final String PAUSELESS_CONSUMPTION_ENABLED = "pauselessConsumptionEnabled"; private PauselessConsumptionUtils() { // Private constructor to prevent instantiation of utility class diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java index ac05d5618e58..04a1dd1a4406 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/BlockingSegmentCompletionFSM.java @@ -106,8 +106,8 @@ public enum BlockingSegmentCompletionFSMState { // time that we need to consider. // We may need to add some time here to allow for getting the lock? For now 0 // We may need to add some time for the committer come back to us (after the build)? For now 0. - private long _maxTimeAllowedToCommitMs; - private final String _controllerVipUrl; + protected long _maxTimeAllowedToCommitMs; + protected final String _controllerVipUrl; public BlockingSegmentCompletionFSM(PinotLLCRealtimeSegmentManager segmentManager, SegmentCompletionManager segmentCompletionManager, LLCSegmentName segmentName, @@ -773,7 +773,7 @@ protected SegmentCompletionProtocol.Response processConsumedAfterCommitStart(Str return response; } - private SegmentCompletionProtocol.Response commitSegment(SegmentCompletionProtocol.Request.Params reqParams, + protected SegmentCompletionProtocol.Response commitSegment(SegmentCompletionProtocol.Request.Params reqParams, CommittingSegmentDescriptor committingSegmentDescriptor) { String instanceId = reqParams.getInstanceId(); StreamPartitionMsgOffset offset = diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java index 7d58a2dd9b92..137a3fb5fe79 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PauselessSegmentCompletionFSM.java @@ -21,6 +21,7 @@ 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; @@ -135,6 +136,52 @@ public SegmentCompletionProtocol.Response extendBuildTime(final String instanceI } } + protected SegmentCompletionProtocol.Response commitSegment(SegmentCompletionProtocol.Request.Params reqParams, + CommittingSegmentDescriptor committingSegmentDescriptor) { + String instanceId = reqParams.getInstanceId(); + StreamPartitionMsgOffset offset = + _streamPartitionMsgOffsetFactory.create(reqParams.getStreamPartitionMsgOffset()); + if (!_state.equals(BlockingSegmentCompletionFSMState.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 = BlockingSegmentCompletionFSMState.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. + // The committingSegmentDescriptor is then updated with the permanent segment location to be saved in metadata + // store. + try { + _segmentManager.commitSegmentFile(_realtimeTableName, committingSegmentDescriptor); + } catch (Exception e) { + _logger.error("Caught exception while committing segment file for segment: {}", _segmentName.getSegmentName(), + e); + return SegmentCompletionProtocol.RESP_FAILED; + } + try { + // Convert to a controller uri if the segment location uses local file scheme. + if (CommonConstants.Segment.LOCAL_SEGMENT_SCHEME + .equalsIgnoreCase(URIUtils.getUri(committingSegmentDescriptor.getSegmentLocation()).getScheme())) { + committingSegmentDescriptor.setSegmentLocation(URIUtils + .constructDownloadUrl(_controllerVipUrl, TableNameBuilder.extractRawTableName(_realtimeTableName), + _segmentName.getSegmentName())); + } + _segmentManager.commitSegmentEndMetadata(_realtimeTableName, committingSegmentDescriptor); + } catch (Exception e) { + _logger + .error("Caught exception while committing segment metadata for segment: {}", _segmentName.getSegmentName(), + e); + return SegmentCompletionProtocol.RESP_FAILED; + } + + _state = BlockingSegmentCompletionFSMState.COMMITTED; + _logger.info("Committed segment {} at offset {} winner {}", _segmentName.getSegmentName(), offset, instanceId); + return SegmentCompletionProtocol.RESP_COMMIT_SUCCESS; + } + + @Override // A common method when the state is > COMMITTER_NOTIFIED. protected SegmentCompletionProtocol.Response processConsumedAfterCommitStart(String instanceId, diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index a4cbfae7a209..3ed88967c67f 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -157,7 +157,8 @@ public class PinotLLCRealtimeSegmentManager { /** * After step 1 of segment completion is done, * this is the max time until which step 3 is allowed to complete. - * See {@link #commitSegmentMetadataInternal(String, CommittingSegmentDescriptor)} for explanation of steps 1 2 3 + * See {@link #commitSegmentMetadataInternal(String, CommittingSegmentDescriptor, boolean)} + * for explanation of steps 1 2 3 * This includes any backoffs and retries for the steps 2 and 3 * The segment will be eligible for repairs by the validation manager, if the time exceeds this value */ @@ -496,35 +497,42 @@ public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescrip committingSegmentDescriptor.setSegmentLocation(uriToMoveTo); } + /** + * This method is invoked after the realtime segment is uploaded but before a response is sent to the server. + * It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore + * records for new segments, and puts them in idealstate in CONSUMING state. + */ + public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { + Preconditions.checkState(!_isStopping, "Segment manager is stopping"); - private void commitSegmentMetadataInternal(String realtimeTableName, - CommittingSegmentDescriptor committingSegmentDescriptor) { - // Validate segment location only for metadata commit - if (StringUtils.isBlank(committingSegmentDescriptor.getSegmentLocation())) { - LOGGER.warn("Committing segment: {} was not uploaded to deep store", - committingSegmentDescriptor.getSegmentName()); - _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); + try { + _numCompletingSegments.addAndGet(1); + // Validate segment location only for metadata commit + if (StringUtils.isBlank(committingSegmentDescriptor.getSegmentLocation())) { + LOGGER.warn("Committing segment: {} was not uploaded to deep store", + committingSegmentDescriptor.getSegmentName()); + _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); + } + commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor, false); + } finally { + _numCompletingSegments.addAndGet(-1); } - commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor, false); } - private void commitSegmentStartMetadataInternal(String realtimeTableName, + private void commitSegmentMetadataInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor, boolean isStartMetadata) { String committingSegmentName = committingSegmentDescriptor.getSegmentName(); - LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); - int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); TableConfig tableConfig = getTableConfig(realtimeTableName); InstancePartitions instancePartitions = getConsumingInstancePartitions(tableConfig); IdealState idealState = getIdealState(realtimeTableName); Preconditions.checkState( idealState.getInstanceStateMap(committingSegmentName).containsValue(SegmentStateModel.CONSUMING), "Failed to find instance in CONSUMING state in IdealState for segment: %s", committingSegmentName); - int numReplicas = getNumReplicas(tableConfig, instancePartitions); /* * Update zookeeper in 3 steps. * - * Step 1: Update PROPERTYSTORE to change the old segment metadata status to COMMITTING + * Step 1: Update PROPERTYSTORE to change the old segment metadata status to COMMITTING/ DONE * Step 2: Update PROPERTYSTORE to create the new segment metadata with status IN_PROGRESS * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state. */ @@ -533,54 +541,26 @@ private void commitSegmentStartMetadataInternal(String realtimeTableName, LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName); long startTimeNs1 = System.nanoTime(); SegmentZKMetadata committingSegmentZKMetadata = - isStartMetadata ? updateCommittingSegmentZKMetadataToCOMMITTING(realtimeTableName, committingSegmentDescriptor) - : updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentDescriptor); - - // Refresh the Broker routing - _helixResourceManager.sendSegmentRefreshMessage(realtimeTableName, committingSegmentName, false, true); + updateCommittingSegmentMetadata(realtimeTableName, committingSegmentDescriptor, isStartMetadata); // Step-2: Create new segment metadata if needed LOGGER.info("Creating new segment metadata with status IN_PROGRESS: {}", committingSegmentName); long startTimeNs2 = System.nanoTime(); - String newConsumingSegmentName = null; - if (!isTablePaused(idealState)) { - List streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig).stream().map( - streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig) - ).collect(Collectors.toList()); - Set partitionIds = getPartitionIds(streamConfigs, idealState); - if (partitionIds.contains(committingSegmentPartitionGroupId)) { - String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName); - long newSegmentCreationTimeMs = getCurrentTimeMs(); - LLCSegmentName newLLCSegment = new LLCSegmentName(rawTableName, committingSegmentPartitionGroupId, - committingLLCSegment.getSequenceNumber() + 1, newSegmentCreationTimeMs); - // TODO: This code does not support size-based segment thresholds for tables with pauseless enabled. The - // calculation of row thresholds based on segment size depends on the size of the previously committed - // segment. For tables with pauseless mode enabled, this size is unavailable at this step because the - // segment has not yet been built. - - createNewSegmentZKMetadata(tableConfig, streamConfigs.get(0), newLLCSegment, newSegmentCreationTimeMs, - committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions, partitionIds.size(), - numReplicas); - newConsumingSegmentName = newLLCSegment.getSegmentName(); - } - } + String newConsumingSegmentName = + createNewSegmentMetadata(tableConfig, idealState, committingSegmentDescriptor, committingSegmentZKMetadata, + instancePartitions); // Step-3: Update IdealState LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName, newConsumingSegmentName); long startTimeNs3 = System.nanoTime(); - SegmentAssignment segmentAssignment = - SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig, _controllerMetrics); - Map instancePartitionsMap = - Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions); // When multiple segments of the same table complete around the same time it is possible that // the idealstate update fails due to contention. We serialize the updates to the idealstate // to reduce this contention. We may still contend with RetentionManager, or other updates // to idealstate from other controllers, but then we have the retry mechanism to get around that. idealState = - updateIdealStateOnSegmentCompletion(realtimeTableName, committingSegmentName, newConsumingSegmentName, - segmentAssignment, instancePartitionsMap); + updateIdealStateForSegments(tableConfig, committingSegmentName, newConsumingSegmentName, instancePartitions); long endTimeNs = System.nanoTime(); LOGGER.info( @@ -608,11 +588,78 @@ private void commitSegmentStartMetadataInternal(String realtimeTableName, } } + // Step 1: Update committing segment metadata + private SegmentZKMetadata updateCommittingSegmentMetadata(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor, boolean isStartMetadata) { + String committingSegmentName = committingSegmentDescriptor.getSegmentName(); + SegmentZKMetadata committingSegmentZKMetadata = + isStartMetadata ? updateCommittingSegmentZKMetadataToCOMMITTING(realtimeTableName, committingSegmentDescriptor) + : updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentDescriptor); + + // Refresh the Broker routing + _helixResourceManager.sendSegmentRefreshMessage(realtimeTableName, committingSegmentName, false, true); + return committingSegmentZKMetadata; + } + + // Step 2: Create new segment metadata + private String createNewSegmentMetadata(TableConfig tableConfig, IdealState idealState, + CommittingSegmentDescriptor committingSegmentDescriptor, + SegmentZKMetadata committingSegmentZKMetadata, InstancePartitions instancePartitions) { + String committingSegmentName = committingSegmentDescriptor.getSegmentName(); + + String realtimeTableName = tableConfig.getTableName(); + int numReplicas = getNumReplicas(tableConfig, instancePartitions); + + String newConsumingSegmentName = null; + if (!isTablePaused(idealState)) { + LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); + int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); + + List streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig).stream().map( + streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig) + ).collect(Collectors.toList()); + Set partitionIds = getPartitionIds(streamConfigs, idealState); + + if (partitionIds.contains(committingSegmentPartitionGroupId)) { + String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName); + long newSegmentCreationTimeMs = getCurrentTimeMs(); + LLCSegmentName newLLCSegment = new LLCSegmentName(rawTableName, committingSegmentPartitionGroupId, + committingLLCSegment.getSequenceNumber() + 1, newSegmentCreationTimeMs); + // TODO: This code does not support size-based segment thresholds for tables with pauseless enabled. The + // calculation of row thresholds based on segment size depends on the size of the previously committed + // segment. For tables with pauseless mode enabled, this size is unavailable at this step because the + // segment has not yet been built. + + createNewSegmentZKMetadata(tableConfig, streamConfigs.get(0), newLLCSegment, newSegmentCreationTimeMs, + committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions, partitionIds.size(), + numReplicas); + newConsumingSegmentName = newLLCSegment.getSegmentName(); + } + } + return newConsumingSegmentName; + } + + // Step 3: Update IdealState + private IdealState updateIdealStateForSegments(TableConfig tableConfig, String committingSegmentName, + String newConsumingSegmentName, InstancePartitions instancePartitions) { + + SegmentAssignment segmentAssignment = + SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig, _controllerMetrics); + Map instancePartitionsMap = + Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions); + + return updateIdealStateOnSegmentCompletion(tableConfig.getTableName(), committingSegmentName, + newConsumingSegmentName, segmentAssignment, instancePartitionsMap); + } + /** - * This method is invoked after the realtime segment is ingested but before the response is sent to the server to - * build the segment. - * It updates the propertystore segment metadata from IN_PROGRESS to COMMITTING, and also creates new propertystore - * records for new segments, and puts them in idealstate in CONSUMING state. + * Invoked during pauseless ingestion after the realtime segment has been ingested but before + * the response is sent to the server to build the segment. + *

+ * This method performs the following actions: + * 1. Updates the property store segment metadata status from IN_PROGRESS to COMMITTING. + * 2. Creates a new property store record for the next consuming segment. + * 3. Updates the ideal state to mark the new segment as CONSUMING. */ public void commitSegmentStartMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { @@ -622,7 +669,37 @@ public void commitSegmentStartMetadata(String realtimeTableName, try { _numCompletingSegments.addAndGet(1); - commitSegmentStartMetadataInternal(realtimeTableName, committingSegmentDescriptor, true); + commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor, true); + } finally { + _numCompletingSegments.addAndGet(-1); + } + } + + /** + * Invoked after the realtime segment has been built and uploaded. + * Updates the metadata like CRC, download URL, etc. in the Zookeeper metadata for the committing segment. + */ + public void commitSegmentEndMetadata(String realtimeTableName, + CommittingSegmentDescriptor committingSegmentDescriptor) { + Preconditions.checkState(!_isStopping, "Segment manager is stopping"); + try { + _numCompletingSegments.addAndGet(1); + // Validate segment location only for metadata commit + if (StringUtils.isBlank(committingSegmentDescriptor.getSegmentLocation())) { + LOGGER.warn("Committing segment: {} was not uploaded to deep store", + committingSegmentDescriptor.getSegmentName()); + _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); + } + String committingSegmentName = committingSegmentDescriptor.getSegmentName(); + Stat stat = new Stat(); + SegmentZKMetadata committingSegmentZKMetadata = + getSegmentZKMetadata(realtimeTableName, committingSegmentName, stat); + Preconditions.checkState(committingSegmentZKMetadata.getStatus() == Status.COMMITTING, + "Segment status for segment %s should be COMMITTING, found: %s", committingSegmentName, + committingSegmentZKMetadata.getStatus()); + LOGGER.info("Updating segment ZK metadata for segment: {}", committingSegmentName); + updateCommittingSegmentMetadata(realtimeTableName, committingSegmentDescriptor, false); + LOGGER.info("Successfully updated segment metadata for segment: {}", committingSegmentName); } finally { _numCompletingSegments.addAndGet(-1); } @@ -634,7 +711,6 @@ public void commitSegmentStartMetadata(String realtimeTableName, private SegmentZKMetadata updateCommittingSegmentZKMetadataToCOMMITTING(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { String segmentName = committingSegmentDescriptor.getSegmentName(); - LOGGER.info("Updating segment ZK metadata for committing segment: {}", segmentName); Stat stat = new Stat(); SegmentZKMetadata committingSegmentZKMetadata = getSegmentZKMetadata(realtimeTableName, segmentName, stat); @@ -650,22 +726,6 @@ private SegmentZKMetadata updateCommittingSegmentZKMetadataToCOMMITTING(String r return committingSegmentZKMetadata; } - /** - * This method is invoked after the realtime segment is uploaded but before a response is sent to the server. - * It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore - * records for new segments, and puts them in idealstate in CONSUMING state. - */ - public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { - Preconditions.checkState(!_isStopping, "Segment manager is stopping"); - - try { - _numCompletingSegments.addAndGet(1); - commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor); - } finally { - _numCompletingSegments.addAndGet(-1); - } - } - /** * Updates segment ZK metadata for the committing segment. @@ -673,13 +733,13 @@ public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDes private SegmentZKMetadata updateCommittingSegmentZKMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { String segmentName = committingSegmentDescriptor.getSegmentName(); - LOGGER.info("Updating segment ZK metadata for committing segment: {}", segmentName); - Stat stat = new Stat(); SegmentZKMetadata committingSegmentZKMetadata = getSegmentZKMetadata(realtimeTableName, segmentName, stat); - Preconditions.checkState(committingSegmentZKMetadata.getStatus() == Status.IN_PROGRESS, - "Segment status for segment: %s should be IN_PROGRESS, found: %s", segmentName, - committingSegmentZKMetadata.getStatus()); + // The segment status can be: + // 1. IN_PROGRESS for normal tables + // 2. COMMITTING for pauseless tables + Preconditions.checkState(committingSegmentZKMetadata.getStatus() != Status.DONE, + "Segment status for segment: %s should not be DONE", segmentName); SegmentMetadataImpl segmentMetadata = committingSegmentDescriptor.getSegmentMetadata(); Preconditions.checkState(segmentMetadata != null, "Failed to find segment metadata from descriptor for segment: %s", segmentName); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java new file mode 100644 index 000000000000..7e899f7c4820 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java @@ -0,0 +1,166 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.common.utils.PauselessConsumptionUtils.PAUSELESS_CONSUMPTION_ENABLED; +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; + + +public class PauselessRealtimeIngestionIntegrationTest extends BaseClusterIntegrationTest { + + private static final int NUM_REALTIME_SEGMENTS = 48; + private static final Logger LOGGER = LoggerFactory.getLogger(PauselessRealtimeIngestionIntegrationTest.class); + private List _avroFiles; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + Schema schema = createSchema(); + addSchema(schema); + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + addTableConfig(tableConfig); + + waitForAllDocsLoaded(600_000L); + } + + @Test(description = "Ensure that all the segments are ingested, built and uploaded when pauseless consumption is " + + "enabled") + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertNoSegmentInProhibitedStatus(segmentZKMetadataList, + CommonConstants.Segment.Realtime.Status.COMMITTING); + }, 1000, 100000, "Some segments have status COMMITTING"); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE + && segmentZKMetadata.getDownloadUrl() == null) { + System.out.println("URl not found for segment: " + segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } + + private boolean assertNoSegmentInProhibitedStatus(List segmentZKMetadataList, + CommonConstants.Segment.Realtime.Status prohibitedStatus) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == prohibitedStatus) { + return false; + } + } + return true; + } + + @Override + protected Map getStreamConfigs() { + Map streamConfigMap = getStreamConfigMap(); + streamConfigMap.put(PAUSELESS_CONSUMPTION_ENABLED, "true"); + streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + return streamConfigMap; + } +} From a97847f586a5ea40fc4c8895d15e4624e5e2c384 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Mon, 30 Dec 2024 13:03:09 +0530 Subject: [PATCH 05/65] Changing the way to enable pausless post introduction of multi-stream ingestion by moving it out of streamConfigMap --- .../utils/PauselessConsumptionUtils.java | 19 +++++++++++++++---- ...elessRealtimeIngestionIntegrationTest.java | 6 ++++-- .../spi/config/table/IndexingConfig.java | 9 +++++++++ .../ingestion/StreamIngestionConfig.java | 11 +++++++++++ 4 files changed, 39 insertions(+), 6 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java index 9aedbbfa180d..e617d9ff13b3 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -18,10 +18,12 @@ */ package org.apache.pinot.common.utils; -import java.util.Map; +import java.util.Optional; import javax.validation.constraints.NotNull; +import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; public class PauselessConsumptionUtils { @@ -40,7 +42,16 @@ private PauselessConsumptionUtils() { * @throws NullPointerException if tableConfig is null */ public static boolean isPauselessEnabled(@NotNull TableConfig tableConfig) { - Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); - return Boolean.parseBoolean(streamConfigMap.getOrDefault(PAUSELESS_CONSUMPTION_ENABLED, "false")); + return checkIngestionConfig(tableConfig) || checkIndexingConfig(tableConfig); + } + + private static boolean checkIndexingConfig(@NotNull TableConfig tableConfig) { + return Optional.ofNullable(tableConfig.getIndexingConfig()).map(IndexingConfig::isPauselessConsumptionEnabled) + .orElse(false); + } + + private static boolean checkIngestionConfig(@NotNull TableConfig tableConfig) { + return Optional.ofNullable(tableConfig.getIngestionConfig()).map(IngestionConfig::getStreamIngestionConfig) + .map(StreamIngestionConfig::isPauselessConsumptionEnabled).orElse(false); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java index 7e899f7c4820..7fc16aa6ce5e 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java @@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils; import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; @@ -43,9 +44,9 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.apache.pinot.common.utils.PauselessConsumptionUtils.PAUSELESS_CONSUMPTION_ENABLED; import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; public class PauselessRealtimeIngestionIntegrationTest extends BaseClusterIntegrationTest { @@ -94,6 +95,7 @@ public void setUp() Schema schema = createSchema(); addSchema(schema); TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); addTableConfig(tableConfig); waitForAllDocsLoaded(600_000L); @@ -105,6 +107,7 @@ public void testSegmentAssignment() throws Exception { String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + assertTrue(PauselessConsumptionUtils.isPauselessEnabled(getRealtimeTableConfig())); TestUtils.waitForCondition((aVoid) -> { List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); return assertNoSegmentInProhibitedStatus(segmentZKMetadataList, @@ -159,7 +162,6 @@ private boolean assertNoSegmentInProhibitedStatus(List segmen @Override protected Map getStreamConfigs() { Map streamConfigMap = getStreamConfigMap(); - streamConfigMap.put(PAUSELESS_CONSUMPTION_ENABLED, "true"); streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); return streamConfigMap; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index 5beb126e0d0e..cfe002480f15 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -66,6 +66,7 @@ public class IndexingConfig extends BaseJsonConfig { private boolean _aggregateMetrics; private boolean _nullHandlingEnabled; private boolean _columnMajorSegmentBuilderEnabled = true; + private boolean _pauselessConsumptionEnabled = false; /** * If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality @@ -459,4 +460,12 @@ public Set getAllReferencedColumns() { } return allColumns; } + + public boolean isPauselessConsumptionEnabled() { + return _pauselessConsumptionEnabled; + } + + public void setPauselessConsumptionEnabled(boolean pauselessConsumptionEnabled) { + _pauselessConsumptionEnabled = pauselessConsumptionEnabled; + } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/StreamIngestionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/StreamIngestionConfig.java index 5b216ca9d2e2..33bdc9c3ce96 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/StreamIngestionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/StreamIngestionConfig.java @@ -40,6 +40,9 @@ public class StreamIngestionConfig extends BaseJsonConfig { @JsonPropertyDescription("Whether to track offsets of the filtered stream messages during consumption.") private boolean _trackFilteredMessageOffsets = false; + @JsonPropertyDescription("Whether pauseless consumption is enabled for the table") + private boolean _pauselessConsumptionEnabled = false; + @JsonCreator public StreamIngestionConfig(@JsonProperty("streamConfigMaps") List> streamConfigMaps) { _streamConfigMaps = streamConfigMaps; @@ -64,4 +67,12 @@ public void setTrackFilteredMessageOffsets(boolean trackFilteredMessageOffsets) public boolean isTrackFilteredMessageOffsets() { return _trackFilteredMessageOffsets; } + + public boolean isPauselessConsumptionEnabled() { + return _pauselessConsumptionEnabled; + } + + public void setPauselessConsumptionEnabled(boolean pauselessConsumptionEnabled) { + _pauselessConsumptionEnabled = pauselessConsumptionEnabled; + } } From 2c2ba863fa4e262ee7e583485fcd2298eb04d015 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Mon, 30 Dec 2024 12:28:56 +0530 Subject: [PATCH 06/65] WIP: Changes in the expected state while performing recovery during pauseless ingestion in RealtimeSegmentValidationManager --- .../realtime/PinotLLCRealtimeSegmentManager.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 3ed88967c67f..40e6bb096788 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -67,6 +67,7 @@ import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.controller.ControllerConf; @@ -1339,7 +1340,7 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, List instanceStateMap = instanceStatesMap.get(latestSegmentName); if (instanceStateMap != null) { // Latest segment of metadata is in idealstate. if (instanceStateMap.containsValue(SegmentStateModel.CONSUMING)) { - if (latestSegmentZKMetadata.getStatus() == Status.DONE) { + if (latestSegmentZKMetadata.getStatus() == statusPostSegmentMetadataUpdate) { - // step-1 of commmitSegmentMetadata is done (i.e. marking old segment as DONE) + // step-1 of commmitSegmentMetadata is done (i.e. marking old segment as DONE/ COMMITTING) // but step-2 is not done (i.e. adding new metadata for the next segment) // and ideal state update (i.e. marking old segment as ONLINE and new segment as CONSUMING) is not done // either. @@ -1449,7 +1456,7 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, List Date: Mon, 30 Dec 2024 16:32:29 +0530 Subject: [PATCH 07/65] Adding changes for ensurePartitionsConsuming function that is utilized by RealtimeSegmentValitdationManager to fix commit protocol failures --- .../segment/SegmentZKMetadataUtils.java | 80 +++++++++++ .../utils/FileUploadDownloadClient.java | 16 +-- .../segment/SegmentZKMetadataUtilsTest.java | 126 +++++++++++++++++ .../PinotLLCRealtimeSegmentManager.java | 128 ++++++++++++++---- .../RealtimeSegmentValidationManager.java | 7 +- .../PinotLLCRealtimeSegmentManagerTest.java | 7 +- .../server/api/resources/TablesResource.java | 82 ++++++++++- 7 files changed, 401 insertions(+), 45 deletions(-) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java create mode 100644 pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java new file mode 100644 index 000000000000..2ffe5cc4422f --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java @@ -0,0 +1,80 @@ +/** + * 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.common.metadata.segment; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.io.IOException; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class SegmentZKMetadataUtils { + private SegmentZKMetadataUtils() { + } + + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentZKMetadataUtils.class); + public static final ObjectMapper MAPPER = createObjectMapper(); + + private static ObjectMapper createObjectMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(SerializationFeature.INDENT_OUTPUT, true); + mapper.configure(MapperFeature.AUTO_DETECT_FIELDS, true); + mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, true); + mapper.configure(MapperFeature.CAN_OVERRIDE_ACCESS_MODIFIERS, true); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return mapper; + } + + public static String serialize(SegmentZKMetadata metadata) throws IOException { + if (metadata == null) { + return null; + } + return MAPPER.writeValueAsString(metadata.toZNRecord()); + } + + public static SegmentZKMetadata deserialize(String jsonString) throws IOException { + if (jsonString == null || jsonString.isEmpty()) { + return null; + } + ObjectNode objectNode = (ObjectNode) MAPPER.readTree(jsonString); + ZNRecord znRecord = MAPPER.treeToValue(objectNode, ZNRecord.class); + return new SegmentZKMetadata(znRecord); + } + + public static SegmentZKMetadata deserialize(ObjectNode objectNode) throws IOException { + if (objectNode == null) { + return null; + } + ZNRecord znRecord = MAPPER.treeToValue(objectNode, ZNRecord.class); + return new SegmentZKMetadata(znRecord); + } + + public static SegmentZKMetadata deserialize(byte[] bytes) throws IOException { + if (bytes == null || bytes.length == 0) { + return null; + } + ZNRecord znRecord = MAPPER.readValue(bytes, ZNRecord.class); + return new SegmentZKMetadata(znRecord); + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 9a60bcc7248c..c23a8d77b44a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -55,9 +55,10 @@ import org.apache.hc.core5.http.message.BasicNameValuePair; import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.restlet.resources.EndReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.StartReplaceSegmentsRequest; -import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.common.utils.http.HttpClientConfig; import org.apache.pinot.spi.auth.AuthProvider; @@ -968,26 +969,25 @@ public String uploadToSegmentStore(String uri) * Used by controllers to send requests to servers: Controller periodic task uses this endpoint to ask servers * to upload committed llc segment to segment store if missing. * @param uri The uri to ask servers to upload segment to segment store - * @return {@link TableLLCSegmentUploadResponse} - segment download url, crc, other metadata + * @return {@link SegmentZKMetadata} - segment download url, crc, other metadata * @throws URISyntaxException * @throws IOException * @throws HttpErrorStatusException */ - public TableLLCSegmentUploadResponse uploadLLCToSegmentStore(String uri) + public SegmentZKMetadata uploadLLCToSegmentStore(String uri) throws URISyntaxException, IOException, HttpErrorStatusException { ClassicRequestBuilder requestBuilder = ClassicRequestBuilder.post(new URI(uri)).setVersion(HttpVersion.HTTP_1_1); // sendRequest checks the response status code SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); - TableLLCSegmentUploadResponse tableLLCSegmentUploadResponse = JsonUtils.stringToObject(response.getResponse(), - TableLLCSegmentUploadResponse.class); - if (tableLLCSegmentUploadResponse.getDownloadUrl() == null - || tableLLCSegmentUploadResponse.getDownloadUrl().isEmpty()) { + SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize(response.getResponse()); + if (segmentZKMetadata.getDownloadUrl() == null + || segmentZKMetadata.getDownloadUrl().isEmpty()) { throw new HttpErrorStatusException( String.format("Returned segment download url is empty after requesting servers to upload by the path: %s", uri), response.getStatusCode()); } - return tableLLCSegmentUploadResponse; + return segmentZKMetadata; } /** diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java new file mode 100644 index 000000000000..6122b997aeb5 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java @@ -0,0 +1,126 @@ +/** + * 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.common.metadata.segment; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + +public class SegmentZKMetadataUtilsTest { + + private SegmentZKMetadata _testMetadata; + private static final String TEST_SEGMENT_NAME = "testSegment"; + private static final String TEST_TABLE_SEGMENT = "mytable__0__0__20220722T2342Z"; + + @BeforeMethod + public void setUp() { + // Create a test metadata object with sample data + _testMetadata = new SegmentZKMetadata(TEST_SEGMENT_NAME); + _testMetadata.setStartTime(1234567890L); + _testMetadata.setEndTime(1234567899L); + _testMetadata.setTimeUnit(TimeUnit.SECONDS); + _testMetadata.setIndexVersion("v1"); + _testMetadata.setTotalDocs(1000); + _testMetadata.setSizeInBytes(1024 * 1024); + _testMetadata.setCrc(123456L); + _testMetadata.setCreationTime(System.currentTimeMillis()); + } + + @Test + public void testSerialize() throws IOException { + // Test successful serialization + String serialized = SegmentZKMetadataUtils.serialize(_testMetadata); + + // Verify basic properties + assertNotNull(serialized, "Serialized string should not be null"); + assertTrue(serialized.contains(TEST_SEGMENT_NAME), "Serialized string should contain segment name"); + assertTrue(serialized.contains("SECONDS"), "Serialized string should contain time unit"); + + // Verify JSON structure + ObjectNode jsonNode = (ObjectNode) SegmentZKMetadataUtils.MAPPER.readTree(serialized); + assertTrue(jsonNode.has("simpleFields"), "Should contain simpleFields"); + assertTrue(jsonNode.has("mapFields"), "Should contain mapFields"); + } + + @Test + public void testSerializeNull() throws IOException { + assertNull(SegmentZKMetadataUtils.serialize(null), "Serializing null should return null"); + } + + @Test + public void testDeserializeString() throws IOException { + String errorStr = "{\"id\":\"" + TEST_TABLE_SEGMENT + "\",\"simpleFields\":" + + "{\"segment.crc\":\"2624963047\",\"segment.creation.time\":\"1658533353347\"," + + "\"segment.download.url\":\"http://localhost:18998/segments/mytable/" + TEST_TABLE_SEGMENT + "\"," + + "\"segment.end.time\":\"1405296000000\",\"segment.flush.threshold.size\":\"2500\"," + + "\"segment.index.version\":\"v3\",\"segment.realtime.endOffset\":\"2500\"," + + "\"segment.realtime.numReplicas\":\"1\",\"segment.realtime.startOffset\":\"0\"," + + "\"segment.realtime.status\":\"DONE\",\"segment.start.time\":\"1404086400000\"," + + "\"segment.time.unit\":\"MILLISECONDS\",\"segment.total.docs\":\"2500\"}," + + "\"mapFields\":{},\"listFields\":{}}"; + + SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize(errorStr); + + // Verify deserialized properties + assertEquals(segmentZKMetadata.getSegmentName(), TEST_TABLE_SEGMENT, + "Segment name should match expected value"); + assertEquals(segmentZKMetadata.getEndTimeMs(), 1405296000000L, + "End time should match expected value"); + assertEquals(segmentZKMetadata.getCrc(), 2624963047L, + "CRC should match expected value"); + } + + @Test + public void testDeserializeObjectNode() throws IOException { + String errorStr = "{\"id\":\"" + TEST_TABLE_SEGMENT + "\",\"simpleFields\":" + + "{\"segment.crc\":\"2624963047\",\"segment.creation.time\":\"1658533353347\"," + + "\"segment.download.url\":\"http://localhost:18998/segments/mytable/" + TEST_TABLE_SEGMENT + "\"," + + "\"segment.end.time\":\"1405296000000\",\"segment.flush.threshold.size\":\"2500\"," + + "\"segment.index.version\":\"v3\",\"segment.realtime.endOffset\":\"2500\"," + + "\"segment.realtime.numReplicas\":\"1\",\"segment.realtime.startOffset\":\"0\"," + + "\"segment.realtime.status\":\"DONE\",\"segment.start.time\":\"1404086400000\"," + + "\"segment.time.unit\":\"MILLISECONDS\",\"segment.total.docs\":\"2500\"}," + + "\"mapFields\":{},\"listFields\":{}}"; + + JsonNode zkChildren = SegmentZKMetadataUtils.MAPPER.readTree(errorStr); + SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize((ObjectNode) zkChildren); + + assertEquals(segmentZKMetadata.getSegmentName(), TEST_TABLE_SEGMENT, + "Segment name should match expected value"); + assertEquals(segmentZKMetadata.getEndTimeMs(), 1405296000000L, + "End time should match expected value"); + } + + @Test + public void testDeserializeBytes() throws IOException { + String serialized = SegmentZKMetadataUtils.serialize(_testMetadata); + byte[] bytes = serialized.getBytes(); + + SegmentZKMetadata deserialized = SegmentZKMetadataUtils.deserialize(bytes); + + assertNotNull(deserialized, "Deserialized object should not be null"); + assertEquals(deserialized.getSegmentName(), _testMetadata.getSegmentName(), + "Segment names should match"); + } +} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 40e6bb096788..60aa7874104f 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -64,7 +64,6 @@ import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.protocols.SegmentCompletionProtocol; -import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.PauselessConsumptionUtils; @@ -677,7 +676,7 @@ public void commitSegmentStartMetadata(String realtimeTableName, } /** - * Invoked after the realtime segment has been built and uploaded. + * Invoked after the realtime segment has been built and uploaded during pauseless ingestion. * Updates the metadata like CRC, download URL, etc. in the Zookeeper metadata for the committing segment. */ public void commitSegmentEndMetadata(String realtimeTableName, @@ -692,6 +691,18 @@ public void commitSegmentEndMetadata(String realtimeTableName, _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.SEGMENT_MISSING_DEEP_STORE_LINK, 1); } String committingSegmentName = committingSegmentDescriptor.getSegmentName(); + // When segment completion begins, the zk metadata and ideal state are updated. + // This is followed by updating zk metadata for the committing segment with crc, size, download url etc. + // during the commit end metadata call. + // We allow only {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms for a segment to + // complete, after which the segment is eligible for repairs by the + // {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager} + if (isExceededMaxSegmentCompletionTime(realtimeTableName, committingSegmentName, getCurrentTimeMs())) { + LOGGER.error("Exceeded max segment completion time. Skipping ZK Metadata update for segment: {}", + committingSegmentName); + throw new HelixHelper.PermanentUpdaterException( + "Exceeded max segment completion time for segment " + committingSegmentName); + } Stat stat = new Stat(); SegmentZKMetadata committingSegmentZKMetadata = getSegmentZKMetadata(realtimeTableName, committingSegmentName, stat); @@ -1383,8 +1394,8 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, List streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig).stream().map( streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig) ).collect(Collectors.toList()); - if (context._runSegmentLevelValidation) { - runSegmentLevelValidation(tableConfig); - } if (shouldEnsureConsuming(tableNameWithType)) { _llcRealtimeSegmentManager.ensureAllPartitionsConsuming(tableConfig, streamConfigs, context._offsetCriteria); } + + if (context._runSegmentLevelValidation) { + runSegmentLevelValidation(tableConfig); + } } /** diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index dbe640d36400..00a64acac276 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -51,7 +51,6 @@ import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; -import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.URIUtils; @@ -1113,9 +1112,11 @@ public void testUploadToSegmentStoreV2() // its final location. This is the expected segment location. String expectedSegmentLocation = segmentManager.createSegmentPath(RAW_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()).toString(); + SegmentZKMetadata segmentZKMetadataCopy = + new SegmentZKMetadata(new ZNRecord(segmentsZKMetadata.get(0).toZNRecord())); + segmentZKMetadataCopy.setDownloadUrl(tempSegmentFileLocation.getPath()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl0)).thenReturn( - new TableLLCSegmentUploadResponse(segmentsZKMetadata.get(0).getSegmentName(), 12345678L, - tempSegmentFileLocation.getPath())); + segmentZKMetadataCopy); // Change 2nd segment status to be DONE, but with default peer download url. // Verify later the download url isn't fixed after upload failure. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 8568a5178c2b..bcd490572aaf 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -33,13 +33,16 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import javax.annotation.Nullable; import javax.inject.Inject; import javax.inject.Named; import javax.ws.rs.DefaultValue; @@ -60,12 +63,13 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.ZKMetadataProvider; +import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; import org.apache.pinot.common.restlet.resources.ServerSegmentsReloadCheckResponse; -import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; import org.apache.pinot.common.restlet.resources.TableSegments; @@ -90,10 +94,13 @@ import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.IndexService; import org.apache.pinot.segment.spi.index.IndexType; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.server.access.AccessControlFactory; import org.apache.pinot.server.api.AdminApiApplication; import org.apache.pinot.server.starter.ServerInstance; @@ -103,6 +110,7 @@ import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.ConsumerPartitionState; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -868,7 +876,7 @@ public String uploadLLCSegment( * @param segmentName name of the segment to be uploaded * @param timeoutMs timeout for the segment upload to the deep-store. If this is negative, the default timeout * would be used. - * @return full url where the segment is uploaded, crc, segmentName. Can add more segment metadata in the future. + * @return full url where the segment is uploaded, crc, segmentName and other segment metadata. * @throws Exception if an error occurred during the segment upload. */ @POST @@ -884,7 +892,7 @@ public String uploadLLCSegment( @ApiResponse(code = 404, message = "Table or segment not found", response = ErrorInfo.class), @ApiResponse(code = 400, message = "Bad request", response = ErrorInfo.class) }) - public TableLLCSegmentUploadResponse uploadLLCSegmentV2( + public String uploadLLCSegmentV2( @ApiParam(value = "Name of the REALTIME table", required = true) @PathParam("realtimeTableNameWithType") String realtimeTableNameWithType, @ApiParam(value = "Name of the segment", required = true) @PathParam("segmentName") String segmentName, @@ -917,8 +925,16 @@ public TableLLCSegmentUploadResponse uploadLLCSegmentV2( String.format("Table %s segment %s does not exist", realtimeTableNameWithType, segmentName), Response.Status.NOT_FOUND); } - String crc = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); - + if (!(segmentDataManager instanceof ImmutableSegmentDataManager)) { + throw new WebApplicationException( + String.format("Table %s segment %s does not exist on the disk", realtimeTableNameWithType, segmentName), + Response.Status.NOT_FOUND); + } + ImmutableSegmentDataManager immutableSegmentDataManager = (ImmutableSegmentDataManager) segmentDataManager; + SegmentMetadataImpl segmentMetadata = + (SegmentMetadataImpl) immutableSegmentDataManager.getSegment().getSegmentMetadata(); + SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(segmentMetadata); + segmentZKMetadata.setSizeInBytes(immutableSegmentDataManager.getSegment().getSegmentSizeBytes()); File segmentTarFile = null; try { // Create the tar.gz segment file in the server's segmentTarUploadDir folder with a unique file name. @@ -948,13 +964,67 @@ public TableLLCSegmentUploadResponse uploadLLCSegmentV2( String.format("Failed to upload table %s segment %s to segment store", realtimeTableNameWithType, segmentName), Response.Status.INTERNAL_SERVER_ERROR); } - return new TableLLCSegmentUploadResponse(segmentName, Long.parseLong(crc), segmentDownloadUrl.toString()); + segmentZKMetadata.setDownloadUrl(segmentDownloadUrl.toString()); + return SegmentZKMetadataUtils.serialize(segmentZKMetadata); } finally { FileUtils.deleteQuietly(segmentTarFile); tableDataManager.releaseSegment(segmentDataManager); } } + private SegmentZKMetadata getSegmentZKMetadata(SegmentMetadataImpl segmentMetadata) { + SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segmentMetadata.getName()); + // set offsets for segment + segmentZKMetadata.setStartOffset(segmentMetadata.getStartOffset()); + segmentZKMetadata.setEndOffset(segmentMetadata.getEndOffset()); + + // The segment is now ONLINE on the server, so marking its status as DONE. + segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); + segmentZKMetadata.setCrc(Long.parseLong(segmentMetadata.getCrc())); + + // set start and end time + // the time unit is always kept to MILLISECONDS when the controller commits the ZK Metadata + if (segmentMetadata.getTotalDocs() > 0) { + Preconditions.checkNotNull(segmentMetadata.getTimeInterval(), + "start/end time information is not correctly written to the segment for table: " + + segmentMetadata.getTableName()); + segmentZKMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis()); + segmentZKMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis()); + } else { + // Set current time as start/end time if total docs is 0 + long now = System.currentTimeMillis(); + segmentZKMetadata.setStartTime(now); + segmentZKMetadata.setEndTime(now); + } + segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); + + SegmentVersion segmentVersion = segmentMetadata.getVersion(); + if (segmentVersion != null) { + segmentZKMetadata.setIndexVersion(segmentVersion.name()); + } + segmentZKMetadata.setTotalDocs(segmentMetadata.getTotalDocs()); + + segmentZKMetadata.setPartitionMetadata(getPartitionMetadataFromSegmentMetadata(segmentMetadata)); + + return segmentZKMetadata; + } + + @Nullable + private SegmentPartitionMetadata getPartitionMetadataFromSegmentMetadata(SegmentMetadataImpl segmentMetadata) { + for (Map.Entry entry : segmentMetadata.getColumnMetadataMap().entrySet()) { + // NOTE: There is at most one partition column. + ColumnMetadata columnMetadata = entry.getValue(); + PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); + if (partitionFunction != null) { + ColumnPartitionMetadata columnPartitionMetadata = + new ColumnPartitionMetadata(partitionFunction.getName(), partitionFunction.getNumPartitions(), + columnMetadata.getPartitions(), columnMetadata.getPartitionFunction().getFunctionConfig()); + return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); + } + } + return null; + } + @GET @Path("tables/{realtimeTableName}/consumingSegmentsInfo") From a041a75c9499031baab8f2ff71e4fc25a7307406 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 30 Dec 2024 17:33:32 +0530 Subject: [PATCH 08/65] Add Server side Reingestion API --- .../api/resources/ReIngestionResource.java | 391 ++++++++++++ .../reingestion/ReIngestionRequest.java | 68 +++ .../reingestion/ReIngestionResponse.java | 36 ++ .../SimpleRealtimeSegmentDataManager.java | 565 ++++++++++++++++++ 4 files changed, 1060 insertions(+) create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java new file mode 100644 index 000000000000..8d69fea9214f --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -0,0 +1,391 @@ +/** + * 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.server.api.resources; + +import com.google.common.base.Function; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiKeyAuthDefinition; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Authorization; +import io.swagger.annotations.SecurityDefinition; +import io.swagger.annotations.SwaggerDefinition; +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.inject.Inject; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.apache.commons.io.FileUtils; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.NameValuePair; +import org.apache.hc.core5.http.message.BasicHeader; +import org.apache.hc.core5.http.message.BasicNameValuePair; +import org.apache.pinot.common.auth.AuthProviderUtils; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.SimpleHttpResponse; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.common.utils.http.HttpClient; +import org.apache.pinot.core.data.manager.InstanceDataManager; +import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; +import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; +import org.apache.pinot.server.starter.ServerInstance; +import org.apache.pinot.spi.auth.AuthProvider; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@Api(tags = "ReIngestion", authorizations = {@Authorization(value = "Bearer")}) +@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name = + HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = "Bearer"))) +@Path("/") +public class ReIngestionResource { + private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); + public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + //TODO: Maximum number of concurrent re-ingestions allowed should be configurable + private static final int MAX_PARALLEL_REINGESTIONS = 10; + + // Map to track ongoing ingestion per segment + private static final ConcurrentHashMap SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + + // Semaphore to enforce global concurrency limit + private static final Semaphore REINGESTION_SEMAPHORE = new Semaphore(MAX_PARALLEL_REINGESTIONS); + + @Inject + private ServerInstance _serverInstance; + + @POST + @Path("/reIngestSegment") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " + + "upload the segment") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, + message = "Internal server error", response = ErrorInfo.class) + }) + public Response reIngestSegment(ReIngestionRequest request) { + try { + String tableNameWithType = request.getTableNameWithType(); + String segmentName = request.getSegmentName(); + + // Try to acquire a permit from the semaphore to ensure we don't exceed max concurrency + if (!REINGESTION_SEMAPHORE.tryAcquire()) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE) + .entity("Too many re-ingestions in progress. Please try again later.") + .build(); + } + + // Check if the segment is already being re-ingested + AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); + if (!isIngesting.compareAndSet(false, true)) { + // The segment is already being ingested + REINGESTION_SEMAPHORE.release(); + return Response.status(Response.Status.CONFLICT) + .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") + .build(); + } + + InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); + if (instanceDataManager == null) { + throw new WebApplicationException(new RuntimeException("Invalid server initialization"), + Response.Status.INTERNAL_SERVER_ERROR); + } + + TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); + if (tableDataManager == null) { + throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); + LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); + + // Get TableConfig and Schema + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + if (tableConfig == null) { + throw new WebApplicationException("Table config not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + Schema schema = indexLoadingConfig.getSchema(); + if (schema == null) { + throw new WebApplicationException("Schema not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + // Fetch SegmentZKMetadata + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (segmentZKMetadata == null) { + throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, + Response.Status.NOT_FOUND); + } + + // Get startOffset, endOffset, partitionGroupId + String startOffsetStr = segmentZKMetadata.getStartOffset(); + String endOffsetStr = segmentZKMetadata.getEndOffset(); + + if (startOffsetStr == null || endOffsetStr == null) { + return Response.serverError().entity("Start offset or end offset is null for segment: " + segmentName).build(); + } + + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + int partitionGroupId = llcSegmentName.getPartitionGroupId(); + + Map streamConfigMap; + try { + streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + } catch (Exception e) { + return Response.serverError().entity("Failed to get stream config for table: " + tableNameWithType).build(); + } + + StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); + + // Set up directories + File resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); + File resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); + + if (!resourceTmpDir.exists()) { + resourceTmpDir.mkdirs(); + } + if (!resourceDataDir.exists()) { + resourceDataDir.mkdirs(); + } + + LOGGER.info("Starting SimpleRealtimeSegmentDataManager..."); + // Instantiate SimpleRealtimeSegmentDataManager + SimpleRealtimeSegmentDataManager manager = + new SimpleRealtimeSegmentDataManager(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, + tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, resourceTmpDir, + resourceDataDir, _serverInstance.getServerMetrics()); + + try { + + manager.startConsumption(); + + waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300000, 0); + + manager.stopConsumption(); + + // After ingestion is complete, get the segment + if (!manager.isSuccess()) { + throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); + } + + + LOGGER.info("Starting build for segment {}", segmentName); + SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + manager.buildSegmentInternal(); + + // Get the segment directory + File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + + if (segmentTarFile == null) { + throw new Exception("Failed to build segment: " + segmentName); + } + + //TODO: Find a way to get auth token here using injection instead of request param + String authToken = request.getAuthToken(); + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(authToken); + List

headers = AuthProviderUtils.toRequestHeaders(authProvider); + + pushSegmentMetadata(tableNameWithType, request.getUploadURI(), segmentTarFile, headers, segmentName); + + LOGGER.info("Re-ingesteed Segment {} uploaded successfully", segmentName); + + } catch (Exception e) { + return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); + } finally { + // Clean up + manager.offload(); + manager.destroy(); + + // Delete temporary directories + FileUtils.deleteQuietly(resourceTmpDir); + FileUtils.deleteQuietly(resourceDataDir); + + isIngesting.set(false); + } + // Return success response + return Response.ok().entity(new ReIngestionResponse("Segment re-ingested and uploaded successfully")).build(); + } catch (Exception e) { + LOGGER.error("Error during re-ingestion", e); + throw new WebApplicationException(e, Response.Status.INTERNAL_SERVER_ERROR); + } finally { + REINGESTION_SEMAPHORE.release(); + } + } + + private void waitForCondition( + Function condition, long checkIntervalMs, long timeoutMs, long gracePeriodMs) { + long endTime = System.currentTimeMillis() + timeoutMs; + + // Adding grace period before starting the condition checks + if (gracePeriodMs > 0) { + LOGGER.info("Waiting for a grace period of {} ms before starting condition checks", gracePeriodMs); + try { + Thread.sleep(gracePeriodMs); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted during grace period wait", e); + } + } + + while (System.currentTimeMillis() < endTime) { + try { + if (Boolean.TRUE.equals(condition.apply(null))) { + LOGGER.info("Condition satisfied: {}", condition); + return; + } + Thread.sleep(checkIntervalMs); + } catch (Exception e) { + throw new RuntimeException("Caught exception while checking the condition", e); + } + } + + throw new RuntimeException("Timeout waiting for condition: " + condition); + } + + /** + * Push segment metadata to the Pinot Controller in METADATA mode. + * + * @param tableNameWithType The table name with type (e.g., "myTable_OFFLINE") + * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") + * @param segmentFile The local segment tar.gz file + * @param authHeaders A map of authentication or additional headers for the request + */ + public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, + List
authHeaders, String segmentName) + throws Exception { + LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), tableNameWithType, + controllerUrl); + String tableName = tableNameWithType; + File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); + + LOGGER.info("Generated segment metadata tar file: {}", segmentMetadataFile.getAbsolutePath()); + try { + // Prepare headers + List
headers = authHeaders; + + // The upload type must be METADATA + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, + FileUploadDownloadClient.FileUploadType.METADATA.toString())); + + // The DOWNLOAD_URI header specifies where the controller can fetch the segment if needed + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentFile.toURI().toString())); + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); + + // Set table name parameter + List parameters = getSegmentPushCommonParams(tableNameWithType); + + // Construct the endpoint URI + URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); + + LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); + + // Perform the metadata upload + SimpleHttpResponse response = + FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, + parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + + LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, tableName, + controllerUrl, response.getStatusCode(), response.getResponse()); + } finally { + FileUtils.deleteQuietly(segmentMetadataFile); + } + } + + private List getSegmentPushCommonParams(String tableNameWithType) { + List params = new ArrayList<>(); + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION, + "true")); + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_NAME, + TableNameBuilder.extractRawTableName(tableNameWithType))); + TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); + if (tableType != null) { + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_TYPE, tableType.toString())); + } else { + throw new RuntimeException(String.format("Failed to determine the tableType from name: %s", tableNameWithType)); + } + return params; + } + + /** + * Generate a tar.gz file containing only the metadata files (metadata.properties, creation.meta) + * from a given Pinot segment tar.gz file. + */ + private File generateSegmentMetadataTar(File segmentTarFile) + throws Exception { + + if (!segmentTarFile.exists()) { + throw new IllegalArgumentException("Segment tar file does not exist: " + segmentTarFile.getAbsolutePath()); + } + + LOGGER.info("Generating segment metadata tar file from segment tar: {}", segmentTarFile.getAbsolutePath()); + File tempDir = Files.createTempDirectory("pinot-segment-temp").toFile(); + String uuid = UUID.randomUUID().toString(); + try { + File metadataDir = new File(tempDir, "segmentMetadataDir-" + uuid); + if (!metadataDir.mkdirs()) { + throw new RuntimeException("Failed to create metadata directory: " + metadataDir.getAbsolutePath()); + } + + LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, + new File(metadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); + + // Extract creation.meta + LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.SEGMENT_CREATION_META, + new File(metadataDir, V1Constants.SEGMENT_CREATION_META)); + + File segmentMetadataFile = + new File(FileUtils.getTempDirectory(), "segmentMetadata-" + UUID.randomUUID() + ".tar.gz"); + TarCompressionUtils.createCompressedTarFile(metadataDir, segmentMetadataFile); + return segmentMetadataFile; + } finally { + FileUtils.deleteQuietly(tempDir); + } + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java new file mode 100644 index 000000000000..bdfc3b23ef2b --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java @@ -0,0 +1,68 @@ +/** + * 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.server.api.resources.reingestion; + +public class ReIngestionRequest { + private String _tableNameWithType; + private String _segmentName; + private String _uploadURI; + private boolean _uploadSegment; + private String _authToken; + + // Getters and setters + public String getTableNameWithType() { + return _tableNameWithType; + } + + public void setTableNameWithType(String tableNameWithType) { + this._tableNameWithType = tableNameWithType; + } + + public String getSegmentName() { + return _segmentName; + } + + public void setSegmentName(String segmentName) { + this._segmentName = segmentName; + } + + public String getUploadURI() { + return _uploadURI; + } + + public void setUploadURI(String uploadURI) { + this._uploadURI = uploadURI; + } + + public boolean isUploadSegment() { + return _uploadSegment; + } + + public void setUploadSegment(boolean uploadSegment) { + _uploadSegment = uploadSegment; + } + + public String getAuthToken() { + return _authToken; + } + + public void setAuthToken(String authToken) { + _authToken = authToken; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java new file mode 100644 index 000000000000..6ecd0774ac3f --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java @@ -0,0 +1,36 @@ +/** + * 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.server.api.resources.reingestion; + +public class ReIngestionResponse { + private String _message; + + public ReIngestionResponse(String message) { + this._message = message; + } + + // Getter and setter + public String getMessage() { + return _message; + } + + public void setMessage(String message) { + this._message = message; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java new file mode 100644 index 000000000000..c7b408c06834 --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -0,0 +1,565 @@ +/** + * 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.server.api.resources.reingestion.utils; + +import com.google.common.annotations.VisibleForTesting; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.segment.local.data.manager.SegmentDataManager; +import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; +import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager; +import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; +import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; +import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; +import org.apache.pinot.segment.local.segment.creator.TransformPipeline; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.utils.IngestionUtils; +import org.apache.pinot.segment.spi.MutableSegment; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; +import org.apache.pinot.spi.config.table.SegmentPartitionConfig; +import org.apache.pinot.spi.config.table.SegmentZKPropsConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.plugin.PluginManager; +import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.StreamConsumerFactory; +import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; +import org.apache.pinot.spi.stream.StreamDataDecoder; +import org.apache.pinot.spi.stream.StreamDataDecoderImpl; +import org.apache.pinot.spi.stream.StreamDataDecoderResult; +import org.apache.pinot.spi.stream.StreamMessage; +import org.apache.pinot.spi.stream.StreamMessageDecoder; +import org.apache.pinot.spi.stream.StreamMetadataProvider; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; +import org.apache.pinot.spi.utils.retry.RetryPolicies; +import org.apache.pinot.spi.utils.retry.RetryPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Simplified Segment Data Manager for ingesting data from a start offset to an end offset. + */ +public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { + + private static final int DEFAULT_CAPACITY = 100_000; + private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; + public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + + private final String _segmentName; + private final String _tableNameWithType; + private final int _partitionGroupId; + private final String _segmentNameStr; + private final SegmentZKMetadata _segmentZKMetadata; + private final TableConfig _tableConfig; + private final Schema _schema; + private final StreamConfig _streamConfig; + private final StreamPartitionMsgOffsetFactory _offsetFactory; + private final StreamConsumerFactory _consumerFactory; + private StreamMetadataProvider _partitionMetadataProvider; + private final PartitionGroupConsumer _consumer; + private final StreamDataDecoder _decoder; + private final MutableSegmentImpl _realtimeSegment; + private final File _resourceTmpDir; + private final File _resourceDataDir; + private final Logger _logger; + private Thread _consumerThread; + private final AtomicBoolean _shouldStop = new AtomicBoolean(false); + private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false); + private final StreamPartitionMsgOffset _startOffset; + private final StreamPartitionMsgOffset _endOffset; + private volatile StreamPartitionMsgOffset _currentOffset; + private volatile int _numRowsIndexed = 0; + private final String _segmentStoreUriStr; + private final int _fetchTimeoutMs; + private final TransformPipeline _transformPipeline; + private volatile boolean _isSuccess = false; + private volatile Throwable _consumptionException; + private final ServerMetrics _serverMetrics; + + public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, + SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, + IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, + File resourceTmpDir, File resourceDataDir, ServerMetrics serverMetrics) + throws Exception { + + _segmentName = segmentName; + _tableNameWithType = tableNameWithType; + _partitionGroupId = partitionGroupId; + _segmentZKMetadata = segmentZKMetadata; + _tableConfig = tableConfig; + _schema = schema; + _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI(); + _streamConfig = streamConfig; + _resourceTmpDir = resourceTmpDir; + _resourceDataDir = resourceDataDir; + _serverMetrics = serverMetrics; + _logger = LoggerFactory.getLogger(SimpleRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); + + _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); + _startOffset = _offsetFactory.create(startOffsetStr); + _endOffset = _offsetFactory.create(endOffsetStr); + + String clientId = getClientId(); + + _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig); + _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId); + _segmentNameStr = _segmentZKMetadata.getSegmentName(); + + // Create a simple PartitionGroupConsumptionStatus + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = + new PartitionGroupConsumptionStatus(_partitionGroupId, 0, _startOffset, null, null); + + _consumer = _consumerFactory.createPartitionGroupConsumer(clientId, partitionGroupConsumptionStatus); + + // Initialize decoder + Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig, _schema); + _decoder = createDecoder(fieldsToRead); + + // Fetch capacity from indexLoadingConfig or use default + int capacity = streamConfig.getFlushThresholdRows(); + if (capacity <= 0) { + capacity = DEFAULT_CAPACITY; + } + + // Fetch average number of multi-values from indexLoadingConfig + int avgNumMultiValues = indexLoadingConfig.getRealtimeAvgMultiValueCount(); + + // Load stats history, here we are using the same stats while as the RealtimeSegmentDataManager so that we are + // much more efficient in allocating buffers. It also works with empty file + String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != null + ? indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + File.separator + _tableNameWithType + : resourceTmpDir.getAbsolutePath(); + File statsHistoryFile = new File(tableDataDir, "segment-stats.ser"); + RealtimeSegmentStatsHistory statsHistory = RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile); + + // Initialize mutable segment with configurations + RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = + new RealtimeSegmentConfig.Builder().setTableNameWithType(_tableNameWithType).setSegmentName(_segmentName) + .setStreamName(_streamConfig.getTopicName()).setSegmentZKMetadata(_segmentZKMetadata) + .setStatsHistory(statsHistory).setSchema(_schema).setCapacity(capacity) + .setAvgNumMultiValues(avgNumMultiValues).setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) + .setConsumerDir(_resourceDataDir.getAbsolutePath()).setMemoryManager( + new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); + + setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); + + _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), _serverMetrics); + + _transformPipeline = new TransformPipeline(tableConfig, schema); + + // Initialize fetch timeout + _fetchTimeoutMs = + _streamConfig.getFetchTimeoutMillis() > 0 ? _streamConfig.getFetchTimeoutMillis() : DEFAULT_FETCH_TIMEOUT_MS; + } + + private String getClientId() { + return _tableNameWithType + "-" + _partitionGroupId; + } + + public void startConsumption() { + // Start the consumer thread + _consumerThread = new Thread(new PartitionConsumer(), _segmentName); + _consumerThread.start(); + } + + private StreamDataDecoder createDecoder(Set fieldsToRead) + throws Exception { + AtomicReference localStreamDataDecoder = new AtomicReference<>(); + RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetryPolicy(5, 1000L, 1.2f); + retryPolicy.attempt(() -> { + try { + StreamMessageDecoder streamMessageDecoder = createMessageDecoder(fieldsToRead); + localStreamDataDecoder.set(new StreamDataDecoderImpl(streamMessageDecoder)); + return true; + } catch (Exception e) { + _logger.warn("Failed to create StreamMessageDecoder. Retrying...", e); + return false; + } + }); + return localStreamDataDecoder.get(); + } + + /** + * Creates a {@link StreamMessageDecoder} using properties in {@link StreamConfig}. + * + * @param fieldsToRead The fields to read from the source stream + * @return The initialized StreamMessageDecoder + */ + private StreamMessageDecoder createMessageDecoder(Set fieldsToRead) { + String decoderClass = _streamConfig.getDecoderClass(); + try { + StreamMessageDecoder decoder = PluginManager.get().createInstance(decoderClass); + decoder.init(fieldsToRead, _streamConfig, _tableConfig, _schema); + return decoder; + } catch (Exception e) { + throw new RuntimeException( + "Caught exception while creating StreamMessageDecoder from stream config: " + _streamConfig, e); + } + } + + private class PartitionConsumer implements Runnable { + @Override + public void run() { + try { + _consumer.start(_startOffset); + _currentOffset = _startOffset; + TransformPipeline.Result reusedResult = new TransformPipeline.Result(); + while (!_shouldStop.get() && _currentOffset.compareTo(_endOffset) < 0) { + // Fetch messages + MessageBatch messageBatch = _consumer.fetchMessages(_currentOffset, _fetchTimeoutMs); + + int messageCount = messageBatch.getMessageCount(); + + for (int i = 0; i < messageCount; i++) { + if (_shouldStop.get()) { + break; + } + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + if (streamMessage.getMetadata() != null && streamMessage.getMetadata().getOffset() != null + && streamMessage.getMetadata().getOffset().compareTo(_endOffset) >= 0) { + _shouldStop.set(true); + _logger.info("Reached end offset: {} for partition group: {}", _endOffset, _partitionGroupId); + break; + } + + // Decode message + StreamDataDecoderResult decodedResult = _decoder.decode(streamMessage); + if (decodedResult.getException() == null) { + // Index message + GenericRow row = decodedResult.getResult(); + + _transformPipeline.processRow(row, reusedResult); + + List transformedRows = reusedResult.getTransformedRows(); + + // TODO: Do enrichment and transforms before indexing + for (GenericRow transformedRow : transformedRows) { + _realtimeSegment.index(transformedRow, streamMessage.getMetadata()); + _numRowsIndexed++; + } + } else { + _logger.warn("Failed to decode message at offset {}: {}", _currentOffset, decodedResult.getException()); + } + } + + _currentOffset = messageBatch.getOffsetOfNextBatch(); + } + _isSuccess = true; + } catch (Exception e) { + _logger.error("Exception in consumer thread", e); + _consumptionException = e; + throw new RuntimeException(e); + } finally { + try { + _consumer.close(); + } catch (Exception e) { + _logger.warn("Failed to close consumer", e); + } + _isDoneConsuming.set(true); + } + } + } + + public void stopConsumption() { + _shouldStop.set(true); + if (_consumerThread.isAlive()) { + _consumerThread.interrupt(); + try { + _consumerThread.join(); + } catch (InterruptedException e) { + _logger.warn("Interrupted while waiting for consumer thread to finish"); + } + } + } + + @Override + public MutableSegment getSegment() { + return _realtimeSegment; + } + + @Override + public String getSegmentName() { + return _segmentName; + } + + @Override + protected void doDestroy() { + _realtimeSegment.destroy(); + } + + @Override + public void doOffload() { + stopConsumption(); + } + + public boolean isDoneConsuming() { + return _isDoneConsuming.get(); + } + + public boolean isSuccess() { + return _isSuccess; + } + + public Throwable getConsumptionException() { + return _consumptionException; + } + + @VisibleForTesting + public SegmentBuildDescriptor buildSegmentInternal() throws Exception { + _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); + final long lockAcquireTimeMillis = now(); + // Build a segment from in-memory rows. + // Use a temporary directory + Path tempSegmentFolder = null; + try { + tempSegmentFolder = + java.nio.file.Files.createTempDirectory(_resourceTmpDir.toPath(), "tmp-" + _segmentNameStr + "-"); + } catch (IOException e) { + _logger.error("Failed to create temporary directory for segment build", e); + return null; + } + + SegmentZKPropsConfig segmentZKPropsConfig = new SegmentZKPropsConfig(); + segmentZKPropsConfig.setStartOffset(_startOffset.toString()); + segmentZKPropsConfig.setEndOffset(_endOffset.toString()); + + // Build the segment + RealtimeSegmentConverter converter = + new RealtimeSegmentConverter(_realtimeSegment, segmentZKPropsConfig, tempSegmentFolder.toString(), + _schema, _tableNameWithType, _tableConfig, _segmentZKMetadata.getSegmentName(), + _tableConfig.getIndexingConfig().isNullHandlingEnabled()); + try { + converter.build(null, _serverMetrics); + } catch (Exception e) { + _logger.error("Failed to build segment", e); + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + return null; + } + final long buildTimeMillis = now() - lockAcquireTimeMillis; + + File dataDir = _resourceDataDir; + File indexDir = new File(dataDir, _segmentNameStr); + FileUtils.deleteQuietly(indexDir); + + File tempIndexDir = new File(tempSegmentFolder.toFile(), _segmentNameStr); + if (!tempIndexDir.exists()) { + _logger.error("Temp index directory {} does not exist", tempIndexDir); + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + return null; + } + try { + FileUtils.moveDirectory(tempIndexDir, indexDir); + } catch (IOException e) { + _logger.error("Caught exception while moving index directory from: {} to: {}", tempIndexDir, indexDir, e); + return null; + } finally { + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + } + + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); + + long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); + File segmentTarFile = new File(dataDir, _segmentNameStr + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); + try { + TarCompressionUtils.createCompressedTarFile(indexDir, segmentTarFile); + } catch (IOException e) { + _logger.error("Caught exception while tarring index directory from: {} to: {}", indexDir, segmentTarFile, e); + return null; + } + + File metadataFile = SegmentDirectoryPaths.findMetadataFile(indexDir); + if (metadataFile == null) { + _logger.error("Failed to find metadata file under index directory: {}", indexDir); + return null; + } + File creationMetaFile = SegmentDirectoryPaths.findCreationMetaFile(indexDir); + if (creationMetaFile == null) { + _logger.error("Failed to find creation meta file under index directory: {}", indexDir); + return null; + } + Map metadataFiles = new HashMap<>(); + metadataFiles.put(V1Constants.MetadataKeys.METADATA_FILE_NAME, metadataFile); + metadataFiles.put(V1Constants.SEGMENT_CREATION_META, creationMetaFile); + return new SegmentBuildDescriptor(segmentTarFile, metadataFiles, _currentOffset, buildTimeMillis, buildTimeMillis, + segmentSizeBytes, segmentMetadata); + } + + protected long now() { + return System.currentTimeMillis(); + } + + public void removeSegmentFile(SegmentBuildDescriptor segmentBuildDescriptor) { + if (segmentBuildDescriptor != null) { + segmentBuildDescriptor.deleteSegmentFile(); + } + } + + /* + * set the following partition parameters in RT segment config builder: + * - partition column + * - partition function + * - partition group id + */ + private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder, + SegmentPartitionConfig segmentPartitionConfig) { + if (segmentPartitionConfig != null) { + Map columnPartitionMap = segmentPartitionConfig.getColumnPartitionMap(); + if (columnPartitionMap.size() == 1) { + Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); + String partitionColumn = entry.getKey(); + ColumnPartitionConfig columnPartitionConfig = entry.getValue(); + String partitionFunctionName = columnPartitionConfig.getFunctionName(); + + // NOTE: Here we compare the number of partitions from the config and the stream, and log a warning and emit a + // metric when they don't match, but use the one from the stream. The mismatch could happen when the + // stream partitions are changed, but the table config has not been updated to reflect the change. + // In such case, picking the number of partitions from the stream can keep the segment properly + // partitioned as long as the partition function is not changed. + int numPartitions = columnPartitionConfig.getNumPartitions(); + try { + // TODO: currentPartitionGroupConsumptionStatus should be fetched from idealState + segmentZkMetadata, + // so that we get back accurate partitionGroups info + // However this is not an issue for Kafka, since partitionGroups never expire and every partitionGroup has + // a single partition + // Fix this before opening support for partitioning in Kinesis + int numPartitionGroups = _partitionMetadataProvider.computePartitionGroupMetadata(getClientId(), _streamConfig, + Collections.emptyList(), /*maxWaitTimeMs=*/5000).size(); + + if (numPartitionGroups != numPartitions) { + _logger.info( + "Number of stream partitions: {} does not match number of partitions in the partition config: {}, " + + "using number of stream " + "partitions", numPartitionGroups, numPartitions); + numPartitions = numPartitionGroups; + } + } catch (Exception e) { + _logger.warn("Failed to get number of stream partitions in 5s, " + + "using number of partitions in the partition config: {}", numPartitions, e); + createPartitionMetadataProvider("Timeout getting number of stream partitions"); + } + + realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); + realtimeSegmentConfigBuilder.setPartitionFunction( + PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, null)); + realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); + } else { + _logger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); + } + } + } + + /** + * Creates a new stream metadata provider + */ + private void createPartitionMetadataProvider(String reason) { + closePartitionMetadataProvider(); + _logger.info("Creating new partition metadata provider, reason: {}", reason); + _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(getClientId(), _partitionGroupId); + } + + private void closePartitionMetadataProvider() { + if (_partitionMetadataProvider != null) { + try { + _partitionMetadataProvider.close(); + } catch (Exception e) { + _logger.warn("Could not close stream metadata provider", e); + } + } + } + + public class SegmentBuildDescriptor { + final File _segmentTarFile; + final Map _metadataFileMap; + final StreamPartitionMsgOffset _offset; + final long _waitTimeMillis; + final long _buildTimeMillis; + final long _segmentSizeBytes; + final SegmentMetadataImpl _segmentMetadata; + + public SegmentBuildDescriptor(@Nullable File segmentTarFile, @Nullable Map metadataFileMap, + StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, SegmentMetadataImpl segmentMetadata) { + _segmentTarFile = segmentTarFile; + _metadataFileMap = metadataFileMap; + _offset = _offsetFactory.create(offset); + _buildTimeMillis = buildTimeMillis; + _waitTimeMillis = waitTimeMillis; + _segmentSizeBytes = segmentSizeBytes; + _segmentMetadata = segmentMetadata; + } + + public StreamPartitionMsgOffset getOffset() { + return _offset; + } + + public long getBuildTimeMillis() { + return _buildTimeMillis; + } + + public long getWaitTimeMillis() { + return _waitTimeMillis; + } + + @Nullable + public File getSegmentTarFile() { + return _segmentTarFile; + } + + @Nullable + public Map getMetadataFiles() { + return _metadataFileMap; + } + + public long getSegmentSizeBytes() { + return _segmentSizeBytes; + } + + public void deleteSegmentFile() { + if (_segmentTarFile != null) { + FileUtils.deleteQuietly(_segmentTarFile); + } + } + + public SegmentMetadataImpl getSegmentMetadata() { + return _segmentMetadata; + } + } +} From b6d090458ac62e64283600a9dd02ce6dd1d1e2a8 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 30 Dec 2024 17:38:56 +0530 Subject: [PATCH 09/65] run segment level validation --- .../controller/validation/RealtimeSegmentValidationManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 3300a0bc9109..dcaf082bcf6e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -176,6 +176,8 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); } + + _llcRealtimeSegmentManager.reIngestSegmentsWithMissingDownloadUrl(tableConfig.getTableName()); } @Override From 58f6c519cdccb81c7849193879d64d833138a1a6 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 30 Dec 2024 18:46:52 +0530 Subject: [PATCH 10/65] Add method to trigger reingestion --- .../utils/FileUploadDownloadClient.java | 52 ++++++++ .../PinotLLCRealtimeSegmentManager.java | 117 ++++++++++++++++++ .../RealtimeSegmentValidationManager.java | 2 +- 3 files changed, 170 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index c23a8d77b44a..1db3e96b0dae 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -126,6 +126,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; + public static final String REINGEST_SEGMENT_PATH = "/reIngestSegment"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); @@ -1248,6 +1249,57 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi httpHeaders, maxStreamRateInByte); } + /** + * Invokes the server's reIngestSegment API via a POST request with JSON payload, + * using Simple HTTP APIs. + * + * POST http://[serverURL]/reIngestSegment + * { + * "tableNameWithType": [tableName], + * "segmentName": [segmentName], + * "uploadURI": [leadControllerUrl], + * "uploadSegment": true + * } + */ + //TODO: Add auth and https support + public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName, + String leadControllerUrl) + throws IOException, URISyntaxException, HttpErrorStatusException { + + String reIngestUrl = String.format(HTTP + "://%s" + REINGEST_SEGMENT_PATH, serverHostPort); + + // Build the JSON payload + Map requestJson = new HashMap<>(); + requestJson.put("tableNameWithType", tableNameWithType); + requestJson.put("segmentName", segmentName); + requestJson.put("uploadURI", leadControllerUrl); + requestJson.put("uploadSegment", true); + + // Convert the request payload to JSON string + String jsonPayload = JsonUtils.objectToString(requestJson); + + // Prepare a POST request with Simple HTTP + ClassicRequestBuilder requestBuilder = ClassicRequestBuilder + .post(new URI(reIngestUrl)) + .setVersion(HttpVersion.HTTP_1_1) + .setHeader("Content-Type", "application/json") + .setHeader("Accept", "application/json") + // Attach our JSON string as the request body + .setEntity(new StringEntity(jsonPayload, ContentType.APPLICATION_JSON)); + + // Send the request using your custom HttpClient wrapper. + // (Adjust the timeout as needed in your environment) + SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( + _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); + + // Check that we got a 2xx response + int statusCode = response.getStatusCode(); + if (statusCode / 100 != 2) { + throw new IOException(String.format("Failed POST to %s, HTTP %d: %s", + reIngestUrl, statusCode, response.getResponse())); + } + } + /** * Generate a param list with a table name attribute. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 60aa7874104f..60eceb3fabdb 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -50,6 +50,7 @@ import org.apache.helix.HelixAdmin; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; +import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.helix.zookeeper.datamodel.ZNRecord; @@ -444,6 +445,10 @@ public IdealState getIdealState(String realtimeTableName) { } } + public ExternalView getExternalView(String realtimeTableName) { + return _helixResourceManager.getTableExternalView(realtimeTableName); + } + @VisibleForTesting void setIdealState(String realtimeTableName, IdealState idealState) { try { @@ -2094,4 +2099,116 @@ String moveSegmentFile(String rawTableName, String segmentName, String segmentLo URI createSegmentPath(String rawTableName, String segmentName) { return URIUtils.getUri(_controllerConf.getDataDir(), rawTableName, URIUtils.encode(segmentName)); } + + /** + * Re-ingests segments that are in DONE status with a missing download URL, but also + * have no peer copy on any server. This method will call the server reIngestSegment API + * on one of the alive servers that are supposed to host that segment according to IdealState. + * + * API signature: + * POST http://[serverURL]/reIngestSegment + * Request body (JSON): + * { + * "tableNameWithType": [tableName], + * "segmentName": [segmentName], + * "uploadURI": [leadControllerUrl], + * "uploadSegment": true + * } + * + * @param tableNameWithType The table name with type, e.g. "myTable_REALTIME" + */ + public void reIngestSegmentsWithErrorState(String tableNameWithType) { + // Step 1: Fetch the ExternalView and all segments + ExternalView externalView = getExternalView(tableNameWithType); + Map> segmentToInstanceStateMap = externalView.getRecord().getMapFields(); + List allSegments = getAllSegments(tableNameWithType); + + // Step 2: For each segment, check the ZK metadata for conditions + for (String segmentName : allSegments) { + // Skip non-LLC segments or segments missing from the ideal state altogether + LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); + if (llcSegmentName == null || !segmentToInstanceStateMap.containsKey(segmentName)) { + continue; + } + + SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); + // We only consider segments that are in COMMITTING which is indicated by having an endOffset + // but have a missing or placeholder download URL + if (segmentZKMetadata.getEndOffset() != null + && isDownloadUrlMissingOrPlaceholder(segmentZKMetadata.getDownloadUrl())) { + + // Step 2a: Check if no peer truly has the segment, i.e. all replicas are ERROR + Map instanceStateMap = segmentToInstanceStateMap.get(segmentName); + boolean allReplicasInError = true; + for (String state : instanceStateMap.values()) { + if (!SegmentStateModel.ERROR.equals(state)) { + allReplicasInError = false; + break; + } + } + + if (!allReplicasInError) { + continue; + } + + // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive + LOGGER.info("Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", + segmentName, tableNameWithType); + + // Find at least one server that should host this segment and is alive + String aliveServer = findAliveServerToReIngest(instanceStateMap.keySet()); + if (aliveServer == null) { + LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, tableNameWithType); + continue; + } + + String leadControllerUrl = getControllerVipUrl(); + try { + _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName, leadControllerUrl); + LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); + } catch (Exception e) { + LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); + } + } + } + } + + /** + * Returns true if the segment downloadUrl is null, empty, or the placeholder for peer download. + */ + private boolean isDownloadUrlMissingOrPlaceholder(String downloadUrl) { + return downloadUrl == null + || downloadUrl.isEmpty() + || CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl); + } + + /** + * Picks one 'alive' server among a set of servers that are supposed to host the segment, + * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. + * This is a simple example; adapt to your environment’s definition of “alive.” + */ + private String findAliveServerToReIngest(Set candidateServers) { + // Get the current live instances from Helix + Set liveInstances = new HashSet<>(_helixAdmin.getInstancesInCluster(_clusterName)); + for (String server : candidateServers) { + if (liveInstances.contains(server)) { + // For a real production check, you might also confirm that HELIX_ENABLED = true, etc. + return extractHostPortFromHelixInstanceId(server); + } + } + return null; + } + + /** + * Example utility to convert a Helix instance ID (like Server_myHost_8098) into host:port for + * building the URL. This depends on your naming conventions. Adjust for your cluster environment. + */ + private String extractHostPortFromHelixInstanceId(String helixInstanceId) { + String[] tokens = helixInstanceId.split("_"); + if (tokens.length >= 3) { + return tokens[1] + ":" + tokens[2]; + } + // Fallback + return helixInstanceId; + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index dcaf082bcf6e..377018095692 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -177,7 +177,7 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); } - _llcRealtimeSegmentManager.reIngestSegmentsWithMissingDownloadUrl(tableConfig.getTableName()); + _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); } @Override From d6313b31b2725c9848d5138e55b72d87da304893 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 30 Dec 2024 19:05:43 +0530 Subject: [PATCH 11/65] Linting fixes --- .../core/realtime/PinotLLCRealtimeSegmentManager.java | 3 ++- .../pinot/server/api/resources/ReIngestionResource.java | 7 +++---- .../api/resources/reingestion/ReIngestionRequest.java | 7 +++---- .../api/resources/reingestion/ReIngestionResponse.java | 4 ++-- .../utils/SimpleRealtimeSegmentDataManager.java | 8 +++++--- 5 files changed, 15 insertions(+), 14 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 60eceb3fabdb..665b182503f1 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2152,7 +2152,8 @@ && isDownloadUrlMissingOrPlaceholder(segmentZKMetadata.getDownloadUrl())) { } // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive - LOGGER.info("Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", + LOGGER.info( + "Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", segmentName, tableNameWithType); // Find at least one server that should host this segment and is alive diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 8d69fea9214f..2e142d3a7a6a 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -233,8 +233,7 @@ public Response reIngestSegment(ReIngestionRequest request) { pushSegmentMetadata(tableNameWithType, request.getUploadURI(), segmentTarFile, headers, segmentName); - LOGGER.info("Re-ingesteed Segment {} uploaded successfully", segmentName); - + LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); } catch (Exception e) { return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); } finally { @@ -298,8 +297,8 @@ private void waitForCondition( public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, List
authHeaders, String segmentName) throws Exception { - LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), tableNameWithType, - controllerUrl); + LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), + tableNameWithType, controllerUrl); String tableName = tableNameWithType; File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java index bdfc3b23ef2b..2e9a63b7cc46 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java @@ -25,13 +25,12 @@ public class ReIngestionRequest { private boolean _uploadSegment; private String _authToken; - // Getters and setters public String getTableNameWithType() { return _tableNameWithType; } public void setTableNameWithType(String tableNameWithType) { - this._tableNameWithType = tableNameWithType; + _tableNameWithType = tableNameWithType; } public String getSegmentName() { @@ -39,7 +38,7 @@ public String getSegmentName() { } public void setSegmentName(String segmentName) { - this._segmentName = segmentName; + _segmentName = segmentName; } public String getUploadURI() { @@ -47,7 +46,7 @@ public String getUploadURI() { } public void setUploadURI(String uploadURI) { - this._uploadURI = uploadURI; + _uploadURI = uploadURI; } public boolean isUploadSegment() { diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java index 6ecd0774ac3f..af1b5f7d55bb 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java @@ -22,7 +22,7 @@ public class ReIngestionResponse { private String _message; public ReIngestionResponse(String message) { - this._message = message; + _message = message; } // Getter and setter @@ -31,6 +31,6 @@ public String getMessage() { } public void setMessage(String message) { - this._message = message; + _message = message; } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java index c7b408c06834..2589d0d18116 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -462,8 +462,9 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen // However this is not an issue for Kafka, since partitionGroups never expire and every partitionGroup has // a single partition // Fix this before opening support for partitioning in Kinesis - int numPartitionGroups = _partitionMetadataProvider.computePartitionGroupMetadata(getClientId(), _streamConfig, - Collections.emptyList(), /*maxWaitTimeMs=*/5000).size(); + int numPartitionGroups = + _partitionMetadataProvider.computePartitionGroupMetadata(getClientId(), _streamConfig, + Collections.emptyList(), /*maxWaitTimeMs=*/5000).size(); if (numPartitionGroups != numPartitions) { _logger.info( @@ -516,7 +517,8 @@ public class SegmentBuildDescriptor { final SegmentMetadataImpl _segmentMetadata; public SegmentBuildDescriptor(@Nullable File segmentTarFile, @Nullable Map metadataFileMap, - StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, SegmentMetadataImpl segmentMetadata) { + StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, + SegmentMetadataImpl segmentMetadata) { _segmentTarFile = segmentTarFile; _metadataFileMap = metadataFileMap; _offset = _offsetFactory.create(offset); From ca6134ab4d4bd078c77a23cf16c79ad2ad6f7f88 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 2 Jan 2025 15:11:48 +0530 Subject: [PATCH 12/65] Adding integration tests for 3 failure scenarios that can occur during commit protocol --- .../helix/core/PinotHelixResourceManager.java | 9 + .../PinotLLCRealtimeSegmentManager.java | 26 ++ .../core/util/FailureInjectionUtils.java | 38 +++ ...IngestionCommitEndMetadataFailureTest.java | 232 +++++++++++++++ ...eIngestionIdealStateUpdateFailureTest.java | 263 ++++++++++++++++++ ...NewSegmentMetadataCreationFailureTest.java | 263 ++++++++++++++++++ 6 files changed, 831 insertions(+) create mode 100644 pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/FailureInjectionUtils.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 77762c2ee8fb..095db4dc4ec5 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -405,6 +405,15 @@ public ZkHelixPropertyStore getPropertyStore() { return _propertyStore; } + /** + * Get the Pinot llc realtime segment manager + * + * @return Pinot llc realtime segment manager + */ + public PinotLLCRealtimeSegmentManager getPinotLLCRealtimeSegmentManager() { + return _pinotLLCRealtimeSegmentManager; + } + /** * Get the linage manager. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 665b182503f1..188e4e79a843 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -84,6 +84,7 @@ import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpdater; import org.apache.pinot.controller.helix.core.retention.strategy.RetentionStrategy; import org.apache.pinot.controller.helix.core.retention.strategy.TimeRetentionStrategy; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; import org.apache.pinot.controller.validation.RealtimeSegmentValidationManager; import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.core.util.PeerServerSegmentFinder; @@ -190,6 +191,8 @@ public class PinotLLCRealtimeSegmentManager { private final AtomicInteger _numCompletingSegments = new AtomicInteger(0); private final ExecutorService _deepStoreUploadExecutor; private final Set _deepStoreUploadExecutorPendingSegments; + @VisibleForTesting + private final Map _failureConfig; private volatile boolean _isStopping = false; @@ -214,6 +217,7 @@ public PinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceMan controllerConf.getDeepStoreRetryUploadParallelism()) : null; _deepStoreUploadExecutorPendingSegments = _isDeepStoreLLCSegmentUploadRetryEnabled ? ConcurrentHashMap.newKeySet() : null; + _failureConfig = new HashMap<>(); } public boolean isDeepStoreLLCSegmentUploadRetryEnabled() { @@ -548,6 +552,11 @@ private void commitSegmentMetadataInternal(String realtimeTableName, SegmentZKMetadata committingSegmentZKMetadata = updateCommittingSegmentMetadata(realtimeTableName, committingSegmentDescriptor, isStartMetadata); + // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the + // segment that encounter failure at this stage of commit protocol. + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION, + _failureConfig); + // Step-2: Create new segment metadata if needed LOGGER.info("Creating new segment metadata with status IN_PROGRESS: {}", committingSegmentName); long startTimeNs2 = System.nanoTime(); @@ -555,6 +564,10 @@ private void commitSegmentMetadataInternal(String realtimeTableName, createNewSegmentMetadata(tableConfig, idealState, committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions); + // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the + // segment that encounter failure at this stage of commit protocol. + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE, _failureConfig); + // Step-3: Update IdealState LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName, newConsumingSegmentName); @@ -686,6 +699,9 @@ public void commitSegmentStartMetadata(String realtimeTableName, */ public void commitSegmentEndMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { + // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the + // segment that encounter failure at this stage of commit protocol. + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA, _failureConfig); Preconditions.checkState(!_isStopping, "Segment manager is stopping"); try { _numCompletingSegments.addAndGet(1); @@ -2212,4 +2228,14 @@ private String extractHostPortFromHelixInstanceId(String helixInstanceId) { // Fallback return helixInstanceId; } + + @VisibleForTesting + public void enableTestFault(String faultType) { + _failureConfig.put(faultType, "true"); + } + + @VisibleForTesting + public void disableTestFault(String faultType) { + _failureConfig.remove(faultType); + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/FailureInjectionUtils.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/FailureInjectionUtils.java new file mode 100644 index 000000000000..2f19a2b9d97d --- /dev/null +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/FailureInjectionUtils.java @@ -0,0 +1,38 @@ +/** + * 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.util; + +import java.util.Map; + + +public class FailureInjectionUtils { + public static final String FAULT_BEFORE_COMMIT_END_METADATA = "FaultBeforeCommitEndMetadata"; + public static final String FAULT_BEFORE_IDEAL_STATE_UPDATE = "FaultBeforeIdealStateUpdate"; + public static final String FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION = "FaultBeforeNewSegmentCreation"; + + private FailureInjectionUtils() { + } + + public static void injectFailure(String faultTypeKey, Map managerConfigs) { + String faultTypeConfig = managerConfigs.getOrDefault(faultTypeKey, "false"); + if (Boolean.parseBoolean(faultTypeConfig)) { + throw new RuntimeException("Injecting failure: " + faultTypeKey); + } + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java new file mode 100644 index 000000000000..a0c3fae7bb97 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java @@ -0,0 +1,232 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + + +public class PauselessRealtimeIngestionCommitEndMetadataFailureTest extends BaseClusterIntegrationTest { + + private static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + private static final Logger LOGGER = + LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); + private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private List _avroFiles; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. + // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + // load data in kafka + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + // create schema for non-pauseless table + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + // add non-pauseless table + TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig2); + + // Ensure that the commit protocol for all the segments have completed before injecting failure + waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + // inject failure in the commit protocol for the pauseless table + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA); + + // create schema for pauseless table + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + // add pauseless table + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); + tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + addTableConfig(tableConfig); + waitForAllDocsLoaded(600_000L); + } + + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + assertUploadUrlEmpty(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType)); + // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can + // run segment level validations. The segment is not fixed by the validation manager in case the desired time + // can not elapsed + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + _controllerStarter.getRealtimeSegmentValidationManager().run(); + // wait for the url to show up after running validation manager + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata(TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); + } + + private void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); + }); + } + + private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } + + private Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private void assertUploadUrlEmpty(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + assertNull(segmentZKMetadata.getDownloadUrl()); + } + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java new file mode 100644 index 000000000000..8e0603035c3d --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java @@ -0,0 +1,263 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + + +public class PauselessRealtimeIngestionIdealStateUpdateFailureTest extends BaseClusterIntegrationTest { + private static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + private static final int NUM_REALTIME_SEGMENTS_WITH_FAILURE = 2; + private static final int NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE = 4; + protected static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; + private static final Logger LOGGER = + LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); + private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private List _avroFiles; + private static boolean _failureEnabled = false; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. + // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + // load data in kafka + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + // create schema for non-pauseless table + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + // add non-pauseless table + TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig2); + + // Ensure that the commit protocol for all the segments have completed before injecting failure + waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + // inject failure in the commit protocol for the pauseless table + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE); + _failureEnabled = true; + + // create schema for pauseless table + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + // add pauseless table + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); + tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + addTableConfig(tableConfig); + waitForAllDocsLoaded(600_000L); + } + + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + String tableNameWithType2 = TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2); + // ensure that the metadata and ideal state only contain 2 segments. + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS_WITH_FAILURE); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return segmentZKMetadataList.size() == NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; + }, 1000, 100000, "New Segment ZK Metadata not created"); + + // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can + // run segment level validations. The segment is not fixed by the validation manager in case the desired time + // can not elapsed + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + _failureEnabled = false; + // inject failure in the commit protocol for the pauseless table + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .disableTestFault(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE); + + // Run validation manager. This should + // 1. Fix url for the segment that failed commit + // 2. Restart ingestion + _controllerStarter.getRealtimeSegmentValidationManager().run(); + // verify all the documents are loaded for both the tables + waitForAllDocsLoaded(600_000L); + waitForDocsLoaded(600_000L, true, tableNameWithType2); + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + verifyIdealState(tableNameWithType2, NUM_REALTIME_SEGMENTS); + // wait for the url to show up after running validation manager + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); + } + + private void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); + }); + } + + protected long getCountStarResult() { + if (_failureEnabled) { + return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; + } + return DEFAULT_COUNT_STAR_RESULT; + } + + private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } + + private Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private void assertUploadUrlEmpty(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + assertNull(segmentZKMetadata.getDownloadUrl()); + } + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java new file mode 100644 index 000000000000..b6c223b3f4b4 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java @@ -0,0 +1,263 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + + +public class PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest extends BaseClusterIntegrationTest { + private static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + private static final int NUM_REALTIME_SEGMENTS_WITH_FAILURE = 2; + private static final int NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE = 2; + protected static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; + private static final Logger LOGGER = + LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); + private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private List _avroFiles; + private static boolean _failureEnabled = false; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. + // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + // load data in kafka + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + // create schema for non-pauseless table + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + // add non-pauseless table + TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig2); + + // Ensure that the commit protocol for all the segments have completed before injecting failure + waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + // inject failure in the commit protocol for the pauseless table + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION); + _failureEnabled = true; + + // create schema for pauseless table + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + // add pauseless table + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); + tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + addTableConfig(tableConfig); + waitForAllDocsLoaded(600_000L); + } + + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + String tableNameWithType2 = TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2); + // ensure that the metadata and ideal state only contain 2 segments. + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS_WITH_FAILURE); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return segmentZKMetadataList.size() == NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; + }, 1000, 100000, "New Segment ZK Metadata not created"); + + // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can + // run segment level validations. The segment is not fixed by the validation manager in case the desired time + // can not elapsed + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + _failureEnabled = false; + // inject failure in the commit protocol for the pauseless table + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .disableTestFault(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION); + + // Run validation manager. This should + // 1. Fix url for the segment that failed commit + // 2. Restart ingestion + _controllerStarter.getRealtimeSegmentValidationManager().run(); + // verify all the documents are loaded for both the tables + waitForAllDocsLoaded(600_000L); + waitForDocsLoaded(600_000L, true, tableNameWithType2); + verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + verifyIdealState(tableNameWithType2, NUM_REALTIME_SEGMENTS); + // wait for the url to show up after running validation manager + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); + } + + private void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); + }); + } + + protected long getCountStarResult() { + if (_failureEnabled) { + return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; + } + return DEFAULT_COUNT_STAR_RESULT; + } + + private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } + + private Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private void assertUploadUrlEmpty(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + assertNull(segmentZKMetadata.getDownloadUrl()); + } + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } +} From 845f616f71e7482f88de839f9f9a7620cb86604b Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 6 Jan 2025 11:14:12 +0530 Subject: [PATCH 13/65] WIP: Reingestion test --- .../utils/FileUploadDownloadClient.java | 16 +- .../PinotLLCRealtimeSegmentManager.java | 18 +- ...timeIngestionSegmentCommitFailureTest.java | 298 ++++++++++++++++++ .../api/resources/ReIngestionResource.java | 17 +- 4 files changed, 336 insertions(+), 13 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 1db3e96b0dae..2b74dca1d1bb 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -1266,7 +1266,15 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, String leadControllerUrl) throws IOException, URISyntaxException, HttpErrorStatusException { - String reIngestUrl = String.format(HTTP + "://%s" + REINGEST_SEGMENT_PATH, serverHostPort); + + if (serverHostPort.contains("http://")) { + serverHostPort = serverHostPort.replace("http://", ""); + } + + String serverHost = serverHostPort.split(":")[0]; + String serverPort = serverHostPort.split(":")[1]; + + URI reIngestUri = getURI(HTTP, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); // Build the JSON payload Map requestJson = new HashMap<>(); @@ -1277,14 +1285,12 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, // Convert the request payload to JSON string String jsonPayload = JsonUtils.objectToString(requestJson); - // Prepare a POST request with Simple HTTP ClassicRequestBuilder requestBuilder = ClassicRequestBuilder - .post(new URI(reIngestUrl)) + .post(reIngestUri) .setVersion(HttpVersion.HTTP_1_1) .setHeader("Content-Type", "application/json") .setHeader("Accept", "application/json") - // Attach our JSON string as the request body .setEntity(new StringEntity(jsonPayload, ContentType.APPLICATION_JSON)); // Send the request using your custom HttpClient wrapper. @@ -1296,7 +1302,7 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, int statusCode = response.getStatusCode(); if (statusCode / 100 != 2) { throw new IOException(String.format("Failed POST to %s, HTTP %d: %s", - reIngestUrl, statusCode, response.getResponse())); + reIngestUri, statusCode, response.getResponse())); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 188e4e79a843..dff7b9d8aa66 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.io.IOException; @@ -52,6 +53,7 @@ import org.apache.helix.InstanceType; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; +import org.apache.helix.model.InstanceConfig; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.assignment.InstancePartitions; @@ -2139,6 +2141,7 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { Map> segmentToInstanceStateMap = externalView.getRecord().getMapFields(); List allSegments = getAllSegments(tableNameWithType); + //TODO: Fetch metadata only for segments that are in ERROR state // Step 2: For each segment, check the ZK metadata for conditions for (String segmentName : allSegments) { // Skip non-LLC segments or segments missing from the ideal state altogether @@ -2206,12 +2209,19 @@ private boolean isDownloadUrlMissingOrPlaceholder(String downloadUrl) { */ private String findAliveServerToReIngest(Set candidateServers) { // Get the current live instances from Helix + // TODO: this line might not be needed Set liveInstances = new HashSet<>(_helixAdmin.getInstancesInCluster(_clusterName)); - for (String server : candidateServers) { - if (liveInstances.contains(server)) { - // For a real production check, you might also confirm that HELIX_ENABLED = true, etc. - return extractHostPortFromHelixInstanceId(server); + try { + BiMap instanceToEndpointMap = + _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); + for (String server : candidateServers) { + if (liveInstances.contains(server)) { + // For a real production check, you might also confirm that HELIX_ENABLED = true, etc. + return extractHostPortFromHelixInstanceId(instanceToEndpointMap.get(server)); + } } + } catch (Exception e) { + LOGGER.warn("Failed to get Helix instance data admin endpoints for servers: {}", candidateServers, e); } return null; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java new file mode 100644 index 000000000000..74bc204240a3 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -0,0 +1,298 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; +import org.apache.pinot.server.starter.helix.BaseServerStarter; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.server.starter.helix.HelixServerStarter; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.JsonUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + + +public class PauselessRealtimeIngestionSegmentCommitFailureTest extends BaseClusterIntegrationTest { + + private static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + private static final Logger LOGGER = + LoggerFactory.getLogger(PauselessRealtimeIngestionSegmentCommitFailureTest.class); + private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private List _avroFiles; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. + // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + // load data in kafka + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + // create schema for non-pauseless table + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + // add non-pauseless table + TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig2); + + // Ensure that the commit protocol for all the segments have completed before injecting failure + waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + // inject failure in the commit protocol for the pauseless table +// _helixResourceManager.getPinotLLCRealtimeSegmentManager() +// .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA); + + // create schema for pauseless table + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + // add pauseless table + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); + tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + addTableConfig(tableConfig); + Thread.sleep(1000L); + waitForDocsLoaded(600_000L, true, getTableName(), getCountStarResult() / 2); +// +// // Make segment data directory read-only to error out the segment commit + makeTableDataDirUnwritable(); + + Thread.sleep(10000L); + +// waitForAllDocsLoaded(600_000L); + } + + private void makeTableDataDirUnwritable() { + File dataDir = _serverStarters.get(0).getServerInstance().getInstanceDataManager() + .getTableDataManager(TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(DEFAULT_TABLE_NAME)) + .getTableDataDir(); + for (File file : dataDir.listFiles()) { + file.setWritable(false, false); + } + dataDir.setWritable(false, false); + + LOGGER.info("Data dir {} is not writable", dataDir.getAbsolutePath()); + } + + private void makeTableDataDirWritable() { + File dataDir = _serverStarters.get(0).getServerInstance().getInstanceDataManager() + .getTableDataManager(TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(DEFAULT_TABLE_NAME)) + .getTableDataDir(); + for (File file : dataDir.listFiles()) { + file.setWritable(true, false); + } + dataDir.setWritable(true, false); + + LOGGER.info("Data dir {} is writable", dataDir.getAbsolutePath()); + } + + protected void waitForDocsLoaded(long timeoutMs, boolean raiseError, String tableName, long numDocs) { + TestUtils.waitForCondition(() -> getCurrentCountStarResult(tableName) >= numDocs, 100L, timeoutMs, + "Failed to load " + numDocs + " documents, found: " + getCurrentCountStarResult(tableName), raiseError, Duration.ofMillis(timeoutMs / 10)); + } + + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); +// verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); + assertTrue(atLeaseOneErrorSegmentInExternalView(tableNameWithType)); +// assertUploadUrlEmpty(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType)); + // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can + // run segment level validations. The segment is not fixed by the validation manager in case the desired time + // can not elapsed + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + makeTableDataDirWritable(); + _controllerStarter.getRealtimeSegmentValidationManager().run(); + // wait for the url to show up after running validation manager + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata(TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); + } + + private void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); + }); + } + + private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } + + private Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private boolean atLeaseOneErrorSegmentInExternalView(String tableName) { + ExternalView resourceEV = _helixResourceManager.getHelixAdmin() + .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); + Map> segmentAssigment = resourceEV.getRecord().getMapFields(); + System.out.println("EXTERNAL VIEW"); + try { + System.out.println(JsonUtils.objectToPrettyString(segmentAssigment)); + } catch (Exception e) { + e.printStackTrace(); + } + for (Map serverToStateMap : segmentAssigment.values()) { + for (String state : serverToStateMap.values()) { + if (state.equals("ERROR")) { + return true; + } + } + } + return false; + } + + private void assertUploadUrlEmpty(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + assertNull(segmentZKMetadata.getDownloadUrl()); + } + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 2e142d3a7a6a..79d2011fb200 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -76,10 +76,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -@Api(tags = "ReIngestion", authorizations = {@Authorization(value = "Bearer")}) -@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name = - HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = "Bearer"))) +import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; +import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; + + +@Api(tags = "ReIngestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), + @Authorization(value = DATABASE)}) +@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = { + @ApiKeyAuthDefinition(name = HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, + key = SWAGGER_AUTHORIZATION_KEY, + description = "The format of the key is ```\"Basic \" or \"Bearer \"```"), + @ApiKeyAuthDefinition(name = DATABASE, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = DATABASE, + description = "Database context passed through http header. If no context is provided 'default' database " + + "context will be considered.")})) @Path("/") public class ReIngestionResource { private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); From d2dd313dc55f04437671a535a48e1f41e5f7f6e0 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 6 Jan 2025 12:32:11 +0530 Subject: [PATCH 14/65] Fix bug with indexes in reingestion --- .../PinotLLCRealtimeSegmentManager.java | 22 ++++++++++++++++--- .../SimpleRealtimeSegmentDataManager.java | 3 ++- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index dff7b9d8aa66..52a0bd8a4e25 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2138,7 +2138,10 @@ URI createSegmentPath(String rawTableName, String segmentName) { public void reIngestSegmentsWithErrorState(String tableNameWithType) { // Step 1: Fetch the ExternalView and all segments ExternalView externalView = getExternalView(tableNameWithType); - Map> segmentToInstanceStateMap = externalView.getRecord().getMapFields(); + IdealState idealState = getIdealState(tableNameWithType); + Map> segmentToInstanceCurrentStateMap = externalView.getRecord().getMapFields(); + Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); + List allSegments = getAllSegments(tableNameWithType); //TODO: Fetch metadata only for segments that are in ERROR state @@ -2146,7 +2149,7 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { for (String segmentName : allSegments) { // Skip non-LLC segments or segments missing from the ideal state altogether LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); - if (llcSegmentName == null || !segmentToInstanceStateMap.containsKey(segmentName)) { + if (llcSegmentName == null || !segmentToInstanceCurrentStateMap.containsKey(segmentName)) { continue; } @@ -2157,7 +2160,20 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { && isDownloadUrlMissingOrPlaceholder(segmentZKMetadata.getDownloadUrl())) { // Step 2a: Check if no peer truly has the segment, i.e. all replicas are ERROR - Map instanceStateMap = segmentToInstanceStateMap.get(segmentName); + Map instanceStateMap = segmentToInstanceCurrentStateMap.get(segmentName); + Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); + + // check if segment is ONLINE in ideal state + boolean isOnline = true; + for (String state : instanceIdealStateMap.values()) { + if (!SegmentStateModel.ONLINE.equals(state)) { + isOnline = false; + break; + } + } + + if (!isOnline) continue; + boolean allReplicasInError = true; for (String state : instanceStateMap.values()) { if (!SegmentStateModel.ERROR.equals(state)) { diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java index 2589d0d18116..ef25eb01379b 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -174,10 +174,11 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith // Initialize mutable segment with configurations RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = - new RealtimeSegmentConfig.Builder().setTableNameWithType(_tableNameWithType).setSegmentName(_segmentName) + new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType).setSegmentName(_segmentName) .setStreamName(_streamConfig.getTopicName()).setSegmentZKMetadata(_segmentZKMetadata) .setStatsHistory(statsHistory).setSchema(_schema).setCapacity(capacity) .setAvgNumMultiValues(avgNumMultiValues).setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) + .setFieldConfigList(tableConfig.getFieldConfigList()) .setConsumerDir(_resourceDataDir.getAbsolutePath()).setMemoryManager( new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); From fb34fc8177c9d9017515b7c228169801fbcd5d1c Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Wed, 8 Jan 2025 16:02:13 +0530 Subject: [PATCH 15/65] Add tests for reingestion --- .../PinotLLCRealtimeSegmentManager.java | 90 ++++++------ .../RealtimeSegmentValidationManager.java | 5 +- .../data/manager/BaseTableDataManager.java | 22 ++- .../realtime/RealtimeSegmentDataManager.java | 4 +- .../realtime/RealtimeTableDataManager.java | 15 +- ...timeIngestionSegmentCommitFailureTest.java | 139 +++++++++--------- ...reInjectingRealtimeSegmentDataManager.java | 67 +++++++++ ...lureInjectingRealtimeTableDataManager.java | 69 +++++++++ ...lureInjectingTableDataManagerProvider.java | 93 ++++++++++++ .../api/resources/ReIngestionResource.java | 53 ++++++- .../SimpleRealtimeSegmentDataManager.java | 13 +- 11 files changed, 435 insertions(+), 135 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 52a0bd8a4e25..2ee0a9e55524 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -49,11 +49,12 @@ import org.apache.helix.ClusterMessagingService; import org.apache.helix.Criteria; import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; +import org.apache.helix.PropertyKey; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; -import org.apache.helix.model.InstanceConfig; import org.apache.helix.store.zk.ZkHelixPropertyStore; import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.assignment.InstancePartitions; @@ -1859,6 +1860,7 @@ private boolean shouldSkipForPauselessMode(String realtimeTableName, SegmentZKMe // Fix the upload URL in case the controller could not finish commit end metadata step of the segment commit // protocol within the required completion time if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING) { + LOGGER.info("Fixing upload URL for segment: {} which is in COMMITTING state", segmentName); return false; } @@ -2142,11 +2144,40 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { Map> segmentToInstanceCurrentStateMap = externalView.getRecord().getMapFields(); Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); - List allSegments = getAllSegments(tableNameWithType); + // find segments in ERROR state in externalView + List segmentsInErrorState = new ArrayList<>(); + for (Map.Entry> entry : segmentToInstanceCurrentStateMap.entrySet()) { + String segmentName = entry.getKey(); + Map instanceStateMap = entry.getValue(); + boolean allReplicasInError = true; + for (String state : instanceStateMap.values()) { + if (!SegmentStateModel.ERROR.equals(state)) { + allReplicasInError = false; + break; + } + } + if (allReplicasInError) { + segmentsInErrorState.add(segmentName); + } + } + + // filter out segments that are not ONLINE in IdealState + for (String segmentName : segmentsInErrorState) { + Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); + boolean isOnline = true; + for (String state : instanceIdealStateMap.values()) { + if (!SegmentStateModel.ONLINE.equals(state)) { + isOnline = false; + break; + } + } + if (!isOnline) { + segmentsInErrorState.remove(segmentName); + } + } - //TODO: Fetch metadata only for segments that are in ERROR state // Step 2: For each segment, check the ZK metadata for conditions - for (String segmentName : allSegments) { + for (String segmentName : segmentsInErrorState) { // Skip non-LLC segments or segments missing from the ideal state altogether LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); if (llcSegmentName == null || !segmentToInstanceCurrentStateMap.containsKey(segmentName)) { @@ -2156,35 +2187,8 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); // We only consider segments that are in COMMITTING which is indicated by having an endOffset // but have a missing or placeholder download URL - if (segmentZKMetadata.getEndOffset() != null - && isDownloadUrlMissingOrPlaceholder(segmentZKMetadata.getDownloadUrl())) { - - // Step 2a: Check if no peer truly has the segment, i.e. all replicas are ERROR - Map instanceStateMap = segmentToInstanceCurrentStateMap.get(segmentName); - Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); - - // check if segment is ONLINE in ideal state - boolean isOnline = true; - for (String state : instanceIdealStateMap.values()) { - if (!SegmentStateModel.ONLINE.equals(state)) { - isOnline = false; - break; - } - } - - if (!isOnline) continue; - - boolean allReplicasInError = true; - for (String state : instanceStateMap.values()) { - if (!SegmentStateModel.ERROR.equals(state)) { - allReplicasInError = false; - break; - } - } - - if (!allReplicasInError) { - continue; - } + if (segmentZKMetadata.getStatus() == Status.COMMITTING) { + Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive LOGGER.info( @@ -2205,28 +2209,26 @@ && isDownloadUrlMissingOrPlaceholder(segmentZKMetadata.getDownloadUrl())) { } catch (Exception e) { LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); } + } else if (segmentZKMetadata.getStatus() == Status.UPLOADED) { + LOGGER.info( + "Segment {} in table {} is in ERROR state with download URL present. Resetting segment to ONLINE state.", + segmentName, tableNameWithType); + _helixResourceManager.resetSegment(tableNameWithType, segmentName, null); } } } - /** - * Returns true if the segment downloadUrl is null, empty, or the placeholder for peer download. - */ - private boolean isDownloadUrlMissingOrPlaceholder(String downloadUrl) { - return downloadUrl == null - || downloadUrl.isEmpty() - || CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD.equals(downloadUrl); - } - /** * Picks one 'alive' server among a set of servers that are supposed to host the segment, * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. * This is a simple example; adapt to your environment’s definition of “alive.” */ + //TODO: Might need to send url registered in DNS instead of host:port private String findAliveServerToReIngest(Set candidateServers) { // Get the current live instances from Helix - // TODO: this line might not be needed - Set liveInstances = new HashSet<>(_helixAdmin.getInstancesInCluster(_clusterName)); + HelixDataAccessor helixDataAccessor = _helixManager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder(); + List liveInstances = helixDataAccessor.getChildNames(keyBuilder.liveInstances()); try { BiMap instanceToEndpointMap = _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 377018095692..1d387223e3f1 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -172,12 +172,13 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { // Update the total document count gauge _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); + _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); +// _llcRealtimeSegmentManager.resetUploadedSegmentsInErrorState(tableConfig.getTableName()); + // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); } - - _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); } @Override diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 3f21d3b7c847..7e0ff11d1369 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -56,6 +56,7 @@ import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; @@ -384,15 +385,20 @@ protected void replaceSegmentIfCrcMismatch(SegmentDataManager segmentDataManager IndexLoadingConfig indexLoadingConfig) throws Exception { String segmentName = segmentDataManager.getSegmentName(); - Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, - "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); - SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); - if (hasSameCRC(zkMetadata, localMetadata)) { - _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); - return; + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + // For pauseless tables, we should replace the segment if download url is missing even if crc is same + if (!PauselessConsumptionUtils.isPauselessEnabled(tableConfig)) { + Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, + "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); + SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); + if (hasSameCRC(zkMetadata, localMetadata)) { + _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); + return; + } + _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, + localMetadata.getCrc(), zkMetadata.getCrc()); } - _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, - localMetadata.getCrc(), zkMetadata.getCrc()); + downloadAndLoadSegment(zkMetadata, indexLoadingConfig); _logger.info("Replaced segment: {} with new CRC: {}", segmentName, zkMetadata.getCrc()); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 848851aa2707..78b9af2e8344 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -1057,7 +1057,7 @@ AtomicBoolean getAcquiredConsumerSemaphore() { } @VisibleForTesting - SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { + protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { // for partial upsert tables, do not release _partitionGroupConsumerSemaphore proactively and rely on offload() // to release the semaphore. This ensures new consuming segment is not consuming until the segment replacement is // complete. @@ -1244,7 +1244,7 @@ protected boolean buildSegmentAndReplace() return true; } - private void closeStreamConsumers() { + protected void closeStreamConsumers() { closePartitionGroupConsumer(); closePartitionMetadataProvider(); if (_acquiredConsumerSemaphore.compareAndSet(true, false)) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index ddadfc11c68d..1bdd3467684e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -544,9 +544,8 @@ private void doAddConsumingSegment(String segmentName) _tableDedupMetadataManager != null ? _tableDedupMetadataManager.getOrCreatePartitionManager(partitionGroupId) : null; RealtimeSegmentDataManager realtimeSegmentDataManager = - new RealtimeSegmentDataManager(zkMetadata, tableConfig, this, _indexDir.getAbsolutePath(), indexLoadingConfig, - schema, llcSegmentName, semaphore, _serverMetrics, partitionUpsertMetadataManager, - partitionDedupMetadataManager, _isTableReadyToConsumeData); + createRealtimeSegmentDataManager(zkMetadata, tableConfig, indexLoadingConfig, schema, llcSegmentName, semaphore, + partitionUpsertMetadataManager, partitionDedupMetadataManager, _isTableReadyToConsumeData); registerSegment(segmentName, realtimeSegmentDataManager, partitionUpsertMetadataManager); if (partitionUpsertMetadataManager != null) { partitionUpsertMetadataManager.trackNewlyAddedSegment(segmentName); @@ -557,6 +556,16 @@ private void doAddConsumingSegment(String segmentName) _logger.info("Added new CONSUMING segment: {}", segmentName); } + protected RealtimeSegmentDataManager createRealtimeSegmentDataManager(SegmentZKMetadata zkMetadata, + TableConfig tableConfig, IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, + Semaphore semaphore, PartitionUpsertMetadataManager partitionUpsertMetadataManager, + PartitionDedupMetadataManager partitionDedupMetadataManager, BooleanSupplier isTableReadyToConsumeData) + throws AttemptsExceededException, RetriableOperationException { + return new RealtimeSegmentDataManager(zkMetadata, tableConfig, this, _indexDir.getAbsolutePath(), + indexLoadingConfig, schema, llcSegmentName, semaphore, _serverMetrics, partitionUpsertMetadataManager, + partitionDedupMetadataManager, isTableReadyToConsumeData); + } + @Override public File downloadSegment(SegmentZKMetadata zkMetadata) throws Exception { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java index 74bc204240a3..fb23f22a50e4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -22,10 +22,12 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import java.time.Duration; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.commons.io.FileUtils; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; @@ -34,16 +36,11 @@ import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; -import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; -import org.apache.pinot.server.starter.helix.BaseServerStarter; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.server.starter.helix.HelixServerStarter; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; import org.slf4j.Logger; @@ -54,8 +51,8 @@ import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; public class PauselessRealtimeIngestionSegmentCommitFailureTest extends BaseClusterIntegrationTest { @@ -90,6 +87,8 @@ protected void overrideServerConf(PinotConfiguration serverConf) { serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, "true"); + serverConf.setProperty("pinot.server.instance." + CommonConstants.Server.TABLE_DATA_MANAGER_PROVIDER_CLASS, + "org.apache.pinot.integration.tests.realtime.FailureInjectingTableDataManagerProvider"); } @BeforeClass @@ -129,10 +128,6 @@ public void setUp() return assertUrlPresent(segmentZKMetadataList); }, 1000, 100000, "Some segments still have missing url"); - // inject failure in the commit protocol for the pauseless table -// _helixResourceManager.getPinotLLCRealtimeSegmentManager() -// .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA); - // create schema for pauseless table schema.setSchemaName(DEFAULT_TABLE_NAME); addSchema(schema); @@ -144,69 +139,81 @@ public void setUp() tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); addTableConfig(tableConfig); - Thread.sleep(1000L); - waitForDocsLoaded(600_000L, true, getTableName(), getCountStarResult() / 2); -// -// // Make segment data directory read-only to error out the segment commit - makeTableDataDirUnwritable(); + Thread.sleep(60000L); + TestUtils.waitForCondition( + (aVoid) -> atLeastOneErrorSegmentInExternalView(TableNameBuilder.REALTIME.tableNameWithType(getTableName())), + 1000, 600000, "Segments still not in error state"); + } - Thread.sleep(10000L); + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); -// waitForAllDocsLoaded(600_000L); - } + // 1) Capture which segments went into the ERROR state + List erroredSegments = getErroredSegmentsInExternalView(tableNameWithType); + assertFalse(erroredSegments.isEmpty(), "No segments found in ERROR state, expected at least one."); - private void makeTableDataDirUnwritable() { - File dataDir = _serverStarters.get(0).getServerInstance().getInstanceDataManager() - .getTableDataManager(TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(DEFAULT_TABLE_NAME)) - .getTableDataDir(); - for (File file : dataDir.listFiles()) { - file.setWritable(false, false); - } - dataDir.setWritable(false, false); + // Let the RealtimeSegmentValidationManager run so it can fix up segments + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + LOGGER.info("Triggering RealtimeSegmentValidationManager to reingest errored segments"); + _controllerStarter.getRealtimeSegmentValidationManager().run(); + LOGGER.info("Finished RealtimeSegmentValidationManager to reingest errored segments"); + + // Wait until there are no ERROR segments in the ExternalView + TestUtils.waitForCondition(aVoid -> { + List errorSegmentsRemaining = getErroredSegmentsInExternalView(tableNameWithType); + return errorSegmentsRemaining.isEmpty(); + }, 10000, 100000, "Some segments are still in ERROR state after resetSegments()"); - LOGGER.info("Data dir {} is not writable", dataDir.getAbsolutePath()); + // Finally compare metadata across your two tables + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata( + TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); } - private void makeTableDataDirWritable() { - File dataDir = _serverStarters.get(0).getServerInstance().getInstanceDataManager() - .getTableDataManager(TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(DEFAULT_TABLE_NAME)) - .getTableDataDir(); - for (File file : dataDir.listFiles()) { - file.setWritable(true, false); + /** + * Returns the list of segment names in ERROR state from the ExternalView of the given table. + */ + private List getErroredSegmentsInExternalView(String tableName) { + ExternalView resourceEV = _helixResourceManager.getHelixAdmin() + .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); + Map> segmentAssignment = resourceEV.getRecord().getMapFields(); + List erroredSegments = new ArrayList<>(); + for (Map.Entry> entry : segmentAssignment.entrySet()) { + String segmentName = entry.getKey(); + Map serverToStateMap = entry.getValue(); + for (String state : serverToStateMap.values()) { + if ("ERROR".equals(state)) { + erroredSegments.add(segmentName); + break; // No need to check other servers for this segment + } + } } - dataDir.setWritable(true, false); - - LOGGER.info("Data dir {} is writable", dataDir.getAbsolutePath()); + return erroredSegments; } - protected void waitForDocsLoaded(long timeoutMs, boolean raiseError, String tableName, long numDocs) { - TestUtils.waitForCondition(() -> getCurrentCountStarResult(tableName) >= numDocs, 100L, timeoutMs, - "Failed to load " + numDocs + " documents, found: " + getCurrentCountStarResult(tableName), raiseError, Duration.ofMillis(timeoutMs / 10)); - } + /** + * Checks that all segments which were previously in ERROR state now have status == UPLOADED. + */ + private boolean haveErroredSegmentsUploaded( + List segmentZKMetadataList, List previouslyErroredSegments) { - @Test - public void testSegmentAssignment() - throws Exception { - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); -// verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); - assertTrue(atLeaseOneErrorSegmentInExternalView(tableNameWithType)); -// assertUploadUrlEmpty(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType)); - // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can - // run segment level validations. The segment is not fixed by the validation manager in case the desired time - // can not elapsed - Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); - makeTableDataDirWritable(); - _controllerStarter.getRealtimeSegmentValidationManager().run(); - // wait for the url to show up after running validation manager - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); + // Convert to a Set for quick lookups + Set erroredSegmentNames = new HashSet<>(previouslyErroredSegments); - compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), - _helixResourceManager.getSegmentsZKMetadata(TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); + for (SegmentZKMetadata metadata : segmentZKMetadataList) { + if (erroredSegmentNames.contains(metadata.getSegmentName())) { + // If it was previously ERROR, then we expect it to have transitioned to UPLOADED + if (metadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { + return false; + } + } + } + return true; } + private void compareZKMetadataForSegments(List segmentsZKMetadata, List segmentsZKMetadata1) { Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); @@ -259,16 +266,10 @@ private void verifyIdealState(String tableName, int numSegmentsExpected) { assertEquals(segmentAssignment.size(), numSegmentsExpected); } - private boolean atLeaseOneErrorSegmentInExternalView(String tableName) { + private boolean atLeastOneErrorSegmentInExternalView(String tableName) { ExternalView resourceEV = _helixResourceManager.getHelixAdmin() .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); - Map> segmentAssigment = resourceEV.getRecord().getMapFields(); - System.out.println("EXTERNAL VIEW"); - try { - System.out.println(JsonUtils.objectToPrettyString(segmentAssigment)); - } catch (Exception e) { - e.printStackTrace(); - } + Map> segmentAssigment = resourceEV.getRecord().getMapFields(); for (Map serverToStateMap : segmentAssigment.values()) { for (String state : serverToStateMap.values()) { if (state.equals("ERROR")) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java new file mode 100644 index 000000000000..475533069494 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java @@ -0,0 +1,67 @@ +/** + * 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.integration.tests.realtime; + +import java.util.concurrent.Semaphore; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; + + +/** + * A specialized RealtimeSegmentDataManager that lets us inject a forced failure + * in the commit step, which occurs strictly after the segmentConsumed message. + */ +public class FailureInjectingRealtimeSegmentDataManager extends RealtimeSegmentDataManager { + + // This flag controls whether commit should forcibly fail. + private final boolean _failCommit; + + /** + * Creates a manager that will forcibly fail the commit segment step. + */ + public FailureInjectingRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, + TableConfig tableConfig, RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, + IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, + Semaphore partitionGroupConsumerSemaphore, ServerMetrics serverMetrics, + boolean failCommit) throws AttemptsExceededException, RetriableOperationException { + // Pass through to the real parent constructor + super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, + indexLoadingConfig, schema, llcSegmentName, partitionGroupConsumerSemaphore, serverMetrics, + null /* no PartitionUpsertMetadataManager */, null /* no PartitionDedupMetadataManager */, + () -> true /* isReadyToConsumeData always true for tests */); + + _failCommit = failCommit; + } + + protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { + if (_failCommit) { + System.out.println("Forcing failure in buildSegmentInternal"); + throw new RuntimeException("Forced failure in buildSegmentInternal"); + } + return super.buildSegmentInternal(forCommit); + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java new file mode 100644 index 000000000000..7dfd703bd98a --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java @@ -0,0 +1,69 @@ +/** + * 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.integration.tests.realtime; + +import java.util.concurrent.Semaphore; +import java.util.function.BooleanSupplier; +import java.util.function.Supplier; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; +import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; +import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.upsert.PartitionUpsertMetadataManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; + + +public class FailureInjectingRealtimeTableDataManager extends RealtimeTableDataManager { + private volatile boolean _hasFailedOnce = false; + + public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore) { + this(segmentBuildSemaphore, () -> true); + } + + public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore, + Supplier isServerReadyToServeQueries) { + super(segmentBuildSemaphore, isServerReadyToServeQueries); + } + + @Override + protected RealtimeSegmentDataManager createRealtimeSegmentDataManager(SegmentZKMetadata zkMetadata, + TableConfig tableConfig, IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, + Semaphore semaphore, PartitionUpsertMetadataManager partitionUpsertMetadataManager, + PartitionDedupMetadataManager partitionDedupMetadataManager, BooleanSupplier isTableReadyToConsumeData) + throws AttemptsExceededException, RetriableOperationException { + + boolean addFailureToCommits = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); + + if (addFailureToCommits) { + if (_hasFailedOnce) { + addFailureToCommits = false; + } else { + _hasFailedOnce = true; + } + } + return new FailureInjectingRealtimeSegmentDataManager(zkMetadata, tableConfig, this, _indexDir.getAbsolutePath(), + indexLoadingConfig, schema, llcSegmentName, semaphore, _serverMetrics, addFailureToCommits); + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java new file mode 100644 index 000000000000..0c323320d51a --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java @@ -0,0 +1,93 @@ +/** + * 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.integration.tests.realtime; + +import com.google.common.cache.Cache; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.helix.HelixManager; +import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; +import org.apache.pinot.core.data.manager.offline.DimensionTableDataManager; +import org.apache.pinot.core.data.manager.offline.OfflineTableDataManager; +import org.apache.pinot.core.data.manager.provider.TableDataManagerProvider; +import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.utils.SegmentLocks; +import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfigProperties; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; + + +/** + * Default implementation of {@link TableDataManagerProvider}. + */ +public class FailureInjectingTableDataManagerProvider implements TableDataManagerProvider { + private InstanceDataManagerConfig _instanceDataManagerConfig; + private HelixManager _helixManager; + private SegmentLocks _segmentLocks; + private Semaphore _segmentBuildSemaphore; + + @Override + public void init(InstanceDataManagerConfig instanceDataManagerConfig, HelixManager helixManager, + SegmentLocks segmentLocks) { + _instanceDataManagerConfig = instanceDataManagerConfig; + _helixManager = helixManager; + _segmentLocks = segmentLocks; + int maxParallelSegmentBuilds = instanceDataManagerConfig.getMaxParallelSegmentBuilds(); + _segmentBuildSemaphore = maxParallelSegmentBuilds > 0 ? new Semaphore(maxParallelSegmentBuilds, true) : null; + } + + @Override + public TableDataManager getTableDataManager(TableConfig tableConfig, @Nullable ExecutorService segmentPreloadExecutor, + @Nullable Cache, SegmentErrorInfo> errorCache, + Supplier isServerReadyToServeQueries) { + TableDataManager tableDataManager; + switch (tableConfig.getTableType()) { + case OFFLINE: + if (tableConfig.isDimTable()) { + tableDataManager = DimensionTableDataManager.createInstanceByTableName(tableConfig.getTableName()); + } else { + tableDataManager = new OfflineTableDataManager(); + } + break; + case REALTIME: + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + if (Boolean.parseBoolean(streamConfigMap.get(StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE)) + && StringUtils.isEmpty(_instanceDataManagerConfig.getSegmentStoreUri())) { + throw new IllegalStateException(String.format("Table has enabled %s config. But the server has not " + + "configured the segmentstore uri. Configure the server config %s", + StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE, CommonConstants.Server.CONFIG_OF_SEGMENT_STORE_URI)); + } + tableDataManager = + new FailureInjectingRealtimeTableDataManager(_segmentBuildSemaphore, isServerReadyToServeQueries); + break; + default: + throw new IllegalStateException(); + } + tableDataManager.init(_instanceDataManagerConfig, _helixManager, _segmentLocks, tableConfig, segmentPreloadExecutor, + errorCache); + return tableDataManager; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 79d2011fb200..7dfe5ce89cc6 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -28,7 +28,11 @@ import io.swagger.annotations.SecurityDefinition; import io.swagger.annotations.SwaggerDefinition; import java.io.File; +import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; import java.util.List; @@ -37,6 +41,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import javax.annotation.Nullable; import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.POST; @@ -52,6 +58,7 @@ import org.apache.hc.core5.http.message.BasicHeader; import org.apache.hc.core5.http.message.BasicNameValuePair; import org.apache.pinot.common.auth.AuthProviderUtils; +import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; @@ -59,6 +66,8 @@ import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; +import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.spi.V1Constants; @@ -71,7 +80,9 @@ import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,7 +234,6 @@ public Response reIngestSegment(ReIngestionRequest request) { throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); } - LOGGER.info("Starting build for segment {}", segmentName); SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = manager.buildSegmentInternal(); @@ -242,6 +252,24 @@ public Response reIngestSegment(ReIngestionRequest request) { pushSegmentMetadata(tableNameWithType, request.getUploadURI(), segmentTarFile, headers, segmentName); + LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); + // wait for segment metadata to have status as UPLOADED + waitForCondition((Void) -> { + SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { + return false; + } + + SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName); + return segmentDataManager instanceof ImmutableSegmentDataManager; + }, 5000, 300000, 0); + + // trigger segment reset call on API + LOGGER.info("Triggering segment reset for uploaded segment {}", segmentName); + HttpClient httpClient = HttpClient.getInstance(); + Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); + resetSegment(httpClient, request.getUploadURI(), tableNameWithType, segmentName, null, headersMap); + LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); } catch (Exception e) { return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); @@ -295,6 +323,29 @@ private void waitForCondition( throw new RuntimeException("Timeout waiting for condition: " + condition); } + public void resetSegment(HttpClient httpClient, String controllerVipUrl, String tableNameWithType, String segmentName, + String targetInstance, Map headers) + throws IOException { + try { + //TODO: send correct headers + HttpClient.wrapAndThrowHttpException(httpClient.sendJsonPostRequest( + new URI(getURLForSegmentReset(controllerVipUrl, tableNameWithType, segmentName, targetInstance)), null, + headers)); + } catch (HttpErrorStatusException | URISyntaxException e) { + throw new IOException(e); + } + } + + private String getURLForSegmentReset(String controllerVipUrl, String tableNameWithType, String segmentName, + @Nullable String targetInstance) { + String query = targetInstance == null ? "reset" : "reset?targetInstance=" + targetInstance; + return StringUtil.join("/", controllerVipUrl, "segments", tableNameWithType, encode(segmentName), query); + } + + private String encode(String s) { + return URLEncoder.encode(s, StandardCharsets.UTF_8); + } + /** * Push segment metadata to the Pinot Controller in METADATA mode. * diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java index ef25eb01379b..23b4b49e111c 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -174,12 +174,13 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith // Initialize mutable segment with configurations RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = - new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType).setSegmentName(_segmentName) - .setStreamName(_streamConfig.getTopicName()).setSegmentZKMetadata(_segmentZKMetadata) - .setStatsHistory(statsHistory).setSchema(_schema).setCapacity(capacity) - .setAvgNumMultiValues(avgNumMultiValues).setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) - .setFieldConfigList(tableConfig.getFieldConfigList()) - .setConsumerDir(_resourceDataDir.getAbsolutePath()).setMemoryManager( + new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType) + .setSegmentName(_segmentName).setStreamName(_streamConfig.getTopicName()) + .setSegmentZKMetadata(_segmentZKMetadata).setStatsHistory(statsHistory).setSchema(_schema) + .setCapacity(capacity).setAvgNumMultiValues(avgNumMultiValues) + .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) + .setFieldConfigList(tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) + .setMemoryManager( new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); From 50725bdd4024b2c7bf62c6627873b476cef85617 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 14 Jan 2025 21:58:11 +0530 Subject: [PATCH 16/65] Fix controller url in Reingestion --- .../utils/FileUploadDownloadClient.java | 2 +- ...erverSegmentCompletionProtocolHandler.java | 8 ++++ .../api/resources/ReIngestionResource.java | 47 +++++++++++++++---- 3 files changed, 46 insertions(+), 11 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 2b74dca1d1bb..41378882f1ec 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -126,7 +126,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; - public static final String REINGEST_SEGMENT_PATH = "/reIngestSegment"; + public static final String REINGEST_SEGMENT_PATH = "/reingestSegment"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java index 6e1e24a688be..593271df0c13 100644 --- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java @@ -100,6 +100,14 @@ public AuthProvider getAuthProvider() { return _authProvider; } + public String getProtocol() { + return _protocol; + } + + public Integer getControllerHttpsPort() { + return _controllerHttpsPort; + } + public SegmentCompletionProtocol.Response segmentCommitStart(SegmentCompletionProtocol.Request.Params params) { SegmentCompletionProtocol.SegmentCommitStartRequest request = new SegmentCompletionProtocol.SegmentCommitStartRequest(params); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 7dfe5ce89cc6..5c145beedcc0 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -35,6 +35,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -53,6 +54,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.NameValuePair; import org.apache.hc.core5.http.message.BasicHeader; @@ -64,6 +66,7 @@ import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.SimpleHttpResponse; import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; @@ -74,6 +77,8 @@ import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; +import org.apache.pinot.server.realtime.ControllerLeaderLocator; +import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.server.starter.ServerInstance; import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.config.table.TableConfig; @@ -88,6 +93,7 @@ import org.slf4j.LoggerFactory; import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; +import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; @@ -117,7 +123,7 @@ public class ReIngestionResource { private ServerInstance _serverInstance; @POST - @Path("/reIngestSegment") + @Path("/reingestSegment") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " @@ -245,12 +251,15 @@ public Response reIngestSegment(ReIngestionRequest request) { throw new Exception("Failed to build segment: " + segmentName); } - //TODO: Find a way to get auth token here using injection instead of request param - String authToken = request.getAuthToken(); - AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(authToken); + ServerSegmentCompletionProtocolHandler protocolHandler = + new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); + + AuthProvider authProvider = protocolHandler.getAuthProvider(); List
headers = AuthProviderUtils.toRequestHeaders(authProvider); - pushSegmentMetadata(tableNameWithType, request.getUploadURI(), segmentTarFile, headers, segmentName); + String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); + + pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); // wait for segment metadata to have status as UPLOADED @@ -268,7 +277,7 @@ public Response reIngestSegment(ReIngestionRequest request) { LOGGER.info("Triggering segment reset for uploaded segment {}", segmentName); HttpClient httpClient = HttpClient.getInstance(); Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); - resetSegment(httpClient, request.getUploadURI(), tableNameWithType, segmentName, null, headersMap); + resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); } catch (Exception e) { @@ -355,7 +364,7 @@ private String encode(String s) { * @param authHeaders A map of authentication or additional headers for the request */ public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, - List
authHeaders, String segmentName) + List
authHeaders, String segmentName, ServerSegmentCompletionProtocolHandler protocolHandler) throws Exception { LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), tableNameWithType, controllerUrl); @@ -384,9 +393,9 @@ public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); // Perform the metadata upload - SimpleHttpResponse response = - FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, - parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + SimpleHttpResponse response = protocolHandler.getFileUploadDownloadClient() + .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, + HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, tableName, controllerUrl, response.getStatusCode(), response.getResponse()); @@ -447,4 +456,22 @@ private File generateSegmentMetadataTar(File segmentTarFile) FileUtils.deleteQuietly(tempDir); } } + + private String getControllerUrl(String rawTableName, ServerSegmentCompletionProtocolHandler protocolHandler) { + ControllerLeaderLocator leaderLocator = ControllerLeaderLocator.getInstance(); + final Pair leaderHostPort = leaderLocator.getControllerLeader(rawTableName); + if (leaderHostPort == null) { + LOGGER.warn("No leader found for table: {}", rawTableName); + return null; + } + Integer port = leaderHostPort.getRight(); + String protocol = protocolHandler.getProtocol(); + Integer controllerHttpsPort = protocolHandler.getControllerHttpsPort(); + if (controllerHttpsPort != null) { + port = controllerHttpsPort; + protocol = HTTPS_PROTOCOL; + } + + return URIUtils.buildURI(protocol, leaderHostPort.getLeft() + ":" + port, "", Collections.emptyMap()).toString(); + } } From e74d360dee9153e1c20c95b0eec75ef86ac07797 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 14 Jan 2025 23:27:03 +0530 Subject: [PATCH 17/65] Support https and auth in reingestion --- .../utils/FileUploadDownloadClient.java | 22 +++++++-------- .../PinotLLCRealtimeSegmentManager.java | 27 +++++++------------ .../reingestion/ReIngestionRequest.java | 27 ------------------- 3 files changed, 18 insertions(+), 58 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 41378882f1ec..9157eda17c8f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -1256,32 +1256,28 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi * POST http://[serverURL]/reIngestSegment * { * "tableNameWithType": [tableName], - * "segmentName": [segmentName], - * "uploadURI": [leadControllerUrl], - * "uploadSegment": true + * "segmentName": [segmentName] * } */ - //TODO: Add auth and https support - public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName, - String leadControllerUrl) + public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName) throws IOException, URISyntaxException, HttpErrorStatusException { - - - if (serverHostPort.contains("http://")) { - serverHostPort = serverHostPort.replace("http://", ""); + String scheme = HTTP; + if (serverHostPort.contains(HTTPS)) { + scheme = HTTPS; + serverHostPort = serverHostPort.replace(HTTPS + "://", ""); + } else if (serverHostPort.contains(HTTP)) { + serverHostPort = serverHostPort.replace(HTTP + "://", ""); } String serverHost = serverHostPort.split(":")[0]; String serverPort = serverHostPort.split(":")[1]; - URI reIngestUri = getURI(HTTP, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); + URI reIngestUri = getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); // Build the JSON payload Map requestJson = new HashMap<>(); requestJson.put("tableNameWithType", tableNameWithType); requestJson.put("segmentName", segmentName); - requestJson.put("uploadURI", leadControllerUrl); - requestJson.put("uploadSegment", true); // Convert the request payload to JSON string String jsonPayload = JsonUtils.objectToString(requestJson); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 2ee0a9e55524..db2525ba54fe 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2202,9 +2202,8 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { continue; } - String leadControllerUrl = getControllerVipUrl(); try { - _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName, leadControllerUrl); + _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName); LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); } catch (Exception e) { LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); @@ -2223,19 +2222,24 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. * This is a simple example; adapt to your environment’s definition of “alive.” */ - //TODO: Might need to send url registered in DNS instead of host:port private String findAliveServerToReIngest(Set candidateServers) { // Get the current live instances from Helix HelixDataAccessor helixDataAccessor = _helixManager.getHelixDataAccessor(); PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder(); List liveInstances = helixDataAccessor.getChildNames(keyBuilder.liveInstances()); try { + // This should ideally handle https scheme as well BiMap instanceToEndpointMap = _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); + + if (instanceToEndpointMap.isEmpty()) { + LOGGER.warn("No instance data admin endpoints found for servers: {}", candidateServers); + return null; + } + for (String server : candidateServers) { if (liveInstances.contains(server)) { - // For a real production check, you might also confirm that HELIX_ENABLED = true, etc. - return extractHostPortFromHelixInstanceId(instanceToEndpointMap.get(server)); + return instanceToEndpointMap.get(server); } } } catch (Exception e) { @@ -2244,19 +2248,6 @@ private String findAliveServerToReIngest(Set candidateServers) { return null; } - /** - * Example utility to convert a Helix instance ID (like Server_myHost_8098) into host:port for - * building the URL. This depends on your naming conventions. Adjust for your cluster environment. - */ - private String extractHostPortFromHelixInstanceId(String helixInstanceId) { - String[] tokens = helixInstanceId.split("_"); - if (tokens.length >= 3) { - return tokens[1] + ":" + tokens[2]; - } - // Fallback - return helixInstanceId; - } - @VisibleForTesting public void enableTestFault(String faultType) { _failureConfig.put(faultType, "true"); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java index 2e9a63b7cc46..a2b92379e549 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java @@ -21,9 +21,6 @@ public class ReIngestionRequest { private String _tableNameWithType; private String _segmentName; - private String _uploadURI; - private boolean _uploadSegment; - private String _authToken; public String getTableNameWithType() { return _tableNameWithType; @@ -40,28 +37,4 @@ public String getSegmentName() { public void setSegmentName(String segmentName) { _segmentName = segmentName; } - - public String getUploadURI() { - return _uploadURI; - } - - public void setUploadURI(String uploadURI) { - _uploadURI = uploadURI; - } - - public boolean isUploadSegment() { - return _uploadSegment; - } - - public void setUploadSegment(boolean uploadSegment) { - _uploadSegment = uploadSegment; - } - - public String getAuthToken() { - return _authToken; - } - - public void setAuthToken(String authToken) { - _authToken = authToken; - } } From d6208a6f82ea25d97cb923b2161172217a87e59c Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Wed, 15 Jan 2025 14:20:31 +0530 Subject: [PATCH 18/65] Removing checks on default crc and replacing them with segment status checks --- .../metadata/segment/SegmentZKMetadata.java | 4 +-- .../PinotLLCRealtimeSegmentManager.java | 29 +++++++++---------- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java index 81e0b48f7d35..69cfe0a63a28 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java @@ -44,8 +44,6 @@ public class SegmentZKMetadata implements ZKMetadata { private boolean _endTimeMsCached; private long _endTimeMs; - public static final long DEFAULT_CRC_VALUE = -1; - public SegmentZKMetadata(String segmentName) { _znRecord = new ZNRecord(segmentName); _simpleFields = _znRecord.getSimpleFields(); @@ -153,7 +151,7 @@ public long getSizeInBytes() { } public long getCrc() { - return _znRecord.getLongField(Segment.CRC, DEFAULT_CRC_VALUE); + return _znRecord.getLongField(Segment.CRC, -1); } public void setCrc(long crc) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 77db5b941851..be13b1bb7215 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -575,10 +575,6 @@ private void commitSegmentMetadataInternal(String realtimeTableName, LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName, newConsumingSegmentName); long startTimeNs3 = System.nanoTime(); - SegmentAssignment segmentAssignment = - SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig, _controllerMetrics); - Map instancePartitionsMap = - Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions); // When multiple segments of the same table complete around the same time it is possible that // the idealstate update fails due to contention. We serialize the updates to the idealstate @@ -1757,8 +1753,7 @@ public void uploadToDeepStoreIfMissing(TableConfig tableConfig, List Date: Wed, 15 Jan 2025 16:09:44 +0530 Subject: [PATCH 19/65] Formatting improvements --- .../metadata/segment/SegmentZKMetadataUtils.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java index 2ffe5cc4422f..d69bb38adc81 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java @@ -25,15 +25,12 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import java.io.IOException; import org.apache.helix.zookeeper.datamodel.ZNRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SegmentZKMetadataUtils { private SegmentZKMetadataUtils() { } - private static final Logger LOGGER = LoggerFactory.getLogger(SegmentZKMetadataUtils.class); public static final ObjectMapper MAPPER = createObjectMapper(); private static ObjectMapper createObjectMapper() { @@ -46,14 +43,16 @@ private static ObjectMapper createObjectMapper() { return mapper; } - public static String serialize(SegmentZKMetadata metadata) throws IOException { + public static String serialize(SegmentZKMetadata metadata) + throws IOException { if (metadata == null) { return null; } return MAPPER.writeValueAsString(metadata.toZNRecord()); } - public static SegmentZKMetadata deserialize(String jsonString) throws IOException { + public static SegmentZKMetadata deserialize(String jsonString) + throws IOException { if (jsonString == null || jsonString.isEmpty()) { return null; } @@ -62,7 +61,8 @@ public static SegmentZKMetadata deserialize(String jsonString) throws IOExceptio return new SegmentZKMetadata(znRecord); } - public static SegmentZKMetadata deserialize(ObjectNode objectNode) throws IOException { + public static SegmentZKMetadata deserialize(ObjectNode objectNode) + throws IOException { if (objectNode == null) { return null; } @@ -70,7 +70,8 @@ public static SegmentZKMetadata deserialize(ObjectNode objectNode) throws IOExce return new SegmentZKMetadata(znRecord); } - public static SegmentZKMetadata deserialize(byte[] bytes) throws IOException { + public static SegmentZKMetadata deserialize(byte[] bytes) + throws IOException { if (bytes == null || bytes.length == 0) { return null; } From 3f05b2f5a226b2d95b648ad152e919c78a0dc1ed Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Wed, 15 Jan 2025 23:23:15 +0530 Subject: [PATCH 20/65] Allowing null table config to be passed for checking pauseless consumption is enabled or not --- .../pinot/common/utils/PauselessConsumptionUtils.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java index 36449a54229f..89b3a2cd2bd5 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PauselessConsumptionUtils.java @@ -19,7 +19,7 @@ package org.apache.pinot.common.utils; import java.util.Optional; -import javax.validation.constraints.NotNull; +import javax.annotation.Nullable; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; @@ -39,8 +39,11 @@ private PauselessConsumptionUtils() { * @return true if pauseless consumption is explicitly enabled, false otherwise * @throws NullPointerException if tableConfig is null */ - public static boolean isPauselessEnabled(@NotNull TableConfig tableConfig) { - return Optional.ofNullable(tableConfig.getIngestionConfig()).map(IngestionConfig::getStreamIngestionConfig) - .map(StreamIngestionConfig::isPauselessConsumptionEnabled).orElse(false); + public static boolean isPauselessEnabled(@Nullable TableConfig tableConfig) { + return Optional.ofNullable(tableConfig) + .map(TableConfig::getIngestionConfig) + .map(IngestionConfig::getStreamIngestionConfig) + .map(StreamIngestionConfig::isPauselessConsumptionEnabled) + .orElse(false); } } From 2012e382e2ef6e3fe6d8fceaac0852345802b519 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 16 Jan 2025 00:14:49 +0530 Subject: [PATCH 21/65] Ensuring upload retries are backward compatible --- .../utils/FileUploadDownloadClient.java | 29 +++- .../PinotLLCRealtimeSegmentManager.java | 141 +++++++++++++----- .../PinotLLCRealtimeSegmentManagerTest.java | 3 +- ...IngestionCommitEndMetadataFailureTest.java | 17 ++- ...eIngestionIdealStateUpdateFailureTest.java | 17 ++- ...NewSegmentMetadataCreationFailureTest.java | 17 ++- .../server/api/resources/TablesResource.java | 104 ++++++++++++- 7 files changed, 284 insertions(+), 44 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 9157eda17c8f..ca864a0a04f0 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -59,6 +59,7 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.restlet.resources.EndReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.StartReplaceSegmentsRequest; +import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.common.utils.http.HttpClientConfig; import org.apache.pinot.spi.auth.AuthProvider; @@ -966,6 +967,32 @@ public String uploadToSegmentStore(String uri) return downloadUrl; } + /** + * Used by controllers to send requests to servers: Controller periodic task uses this endpoint to ask servers + * to upload committed llc segment to segment store if missing. + * @param uri The uri to ask servers to upload segment to segment store + * @return {@link TableLLCSegmentUploadResponse} - segment download url, crc, other metadata + * @throws URISyntaxException + * @throws IOException + * @throws HttpErrorStatusException + */ + public TableLLCSegmentUploadResponse uploadLLCToSegmentStore(String uri) + throws URISyntaxException, IOException, HttpErrorStatusException { + ClassicRequestBuilder requestBuilder = ClassicRequestBuilder.post(new URI(uri)).setVersion(HttpVersion.HTTP_1_1); + // sendRequest checks the response status code + SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( + _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); + TableLLCSegmentUploadResponse tableLLCSegmentUploadResponse = JsonUtils.stringToObject(response.getResponse(), + TableLLCSegmentUploadResponse.class); + if (tableLLCSegmentUploadResponse.getDownloadUrl() == null + || tableLLCSegmentUploadResponse.getDownloadUrl().isEmpty()) { + throw new HttpErrorStatusException( + String.format("Returned segment download url is empty after requesting servers to upload by the path: %s", + uri), response.getStatusCode()); + } + return tableLLCSegmentUploadResponse; + } + /** * Used by controllers to send requests to servers: Controller periodic task uses this endpoint to ask servers * to upload committed llc segment to segment store if missing. @@ -975,7 +1002,7 @@ public String uploadToSegmentStore(String uri) * @throws IOException * @throws HttpErrorStatusException */ - public SegmentZKMetadata uploadLLCToSegmentStore(String uri) + public SegmentZKMetadata uploadLLCToSegmentStoreWithZKMetadata(String uri) throws URISyntaxException, IOException, HttpErrorStatusException { ClassicRequestBuilder requestBuilder = ClassicRequestBuilder.post(new URI(uri)).setVersion(HttpVersion.HTTP_1_1); // sendRequest checks the response status code diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index be13b1bb7215..51b2f0c17e32 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -68,6 +68,7 @@ import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.protocols.SegmentCompletionProtocol; +import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.PauselessConsumptionUtils; @@ -119,7 +120,6 @@ import org.apache.pinot.spi.utils.CommonConstants.Segment.Realtime.Status; import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.JsonUtils; -import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.TimeUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.spi.utils.retry.RetryPolicies; @@ -1736,39 +1736,9 @@ public void uploadToDeepStoreIfMissing(TableConfig tableConfig, List uploadAttempts = Arrays.asList( + // Primary method + () -> { + String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegmentToDeepStore"); + LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, + serverUploadRequestUrl); + SegmentZKMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( + serverUploadRequestUrl); + handleMetadataUpload(segmentName, rawTableName, segmentZKMetadata, uploadedMetadata, pinotFS); + }, + // First fallback + () -> { + String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegment"); + LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, + serverUploadRequestUrl); + TableLLCSegmentUploadResponse response = + _fileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl); + handleLLCUpload(segmentName, rawTableName, segmentZKMetadata, response, pinotFS); + }, + // Legacy fallback + () -> { + String serverUploadRequestUrl = getUploadUrl(uri, "upload"); + LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, + serverUploadRequestUrl); + + String tempUrl = _fileUploadDownloadClient.uploadToSegmentStore(serverUploadRequestUrl); + handleBasicUpload(segmentName, rawTableName, segmentZKMetadata, tempUrl, pinotFS); + } + ); + + // Try each method in sequence until one succeeds + Exception lastException = null; + for (UploadAttempt attempt : uploadAttempts) { + try { + attempt.upload(); + return; // Success, exit the method + } catch (Exception e) { + lastException = e; + LOGGER.warn("Upload attempt failed for segment {}, trying next method", segmentName, e); + } + } + + // All attempts for segment upload failed + throw new Exception("All upload attempts failed for segment " + segmentName, lastException); + } + + private String getUploadUrl(URI uri, String endpoint) { + return String.format("%s/%s?uploadTimeoutMs=%d", + uri.toString(), endpoint, _deepstoreUploadRetryTimeoutMs); + } + + private void handleMetadataUpload(String segmentName, String rawTableName, + SegmentZKMetadata currentMetadata, SegmentZKMetadata uploadedMetadata, + PinotFS pinotFS) + throws Exception { + + String downloadUrl = moveSegmentFile(rawTableName, segmentName, uploadedMetadata.getDownloadUrl(), pinotFS); + LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); + currentMetadata.setDownloadUrl(downloadUrl); + + if (uploadedMetadata.getCrc() != currentMetadata.getCrc()) { + LOGGER.info("Updating segment {} crc in ZK to be {} from previous {}", segmentName, + uploadedMetadata.getCrc(), currentMetadata.getCrc()); + updateSegmentMetadata(currentMetadata, uploadedMetadata); + } + } + + private void handleLLCUpload(String segmentName, String rawTableName, + SegmentZKMetadata currentMetadata, TableLLCSegmentUploadResponse response, + PinotFS pinotFS) + throws Exception { + + String downloadUrl = moveSegmentFile(rawTableName, segmentName, response.getDownloadUrl(), pinotFS); + LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); + currentMetadata.setDownloadUrl(downloadUrl); + + if (response.getCrc() != currentMetadata.getCrc()) { + LOGGER.info("Updating segment {} crc in ZK to be {} from previous {}", segmentName, + response.getCrc(), currentMetadata.getCrc()); + currentMetadata.setCrc(response.getCrc()); + } + } + + private void handleBasicUpload(String segmentName, String rawTableName, + SegmentZKMetadata metadata, String tempDownloadUrl, PinotFS pinotFS) + throws Exception { + + String downloadUrl = moveSegmentFile(rawTableName, segmentName, tempDownloadUrl, pinotFS); + metadata.setDownloadUrl(downloadUrl); + } + /** * Updates the segment metadata in ZooKeeper with information from the uploaded segment. * diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 00a64acac276..4980c6fe9aeb 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -1115,7 +1115,8 @@ public void testUploadToSegmentStoreV2() SegmentZKMetadata segmentZKMetadataCopy = new SegmentZKMetadata(new ZNRecord(segmentsZKMetadata.get(0).toZNRecord())); segmentZKMetadataCopy.setDownloadUrl(tempSegmentFileLocation.getPath()); - when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl0)).thenReturn( + when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( + serverUploadRequestUrl0)).thenReturn( segmentZKMetadataCopy); // Change 2nd segment status to be DONE, but with default peer download url. diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java index a0c3fae7bb97..65d6d78bff14 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java @@ -35,6 +35,8 @@ import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; @@ -134,8 +136,19 @@ public void setUp() TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); - tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + + // Replace stream config from indexing config to ingestion config + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0) + .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); waitForAllDocsLoaded(600_000L); } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java index 8e0603035c3d..1d3d846c7bb7 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java @@ -35,6 +35,8 @@ import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; @@ -138,8 +140,19 @@ public void setUp() TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); - tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + + // Replace stream config from indexing config to ingestion config + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0) + .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); waitForAllDocsLoaded(600_000L); } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java index b6c223b3f4b4..34b2fbb4b9d7 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java @@ -35,6 +35,8 @@ import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; @@ -138,8 +140,19 @@ public void setUp() TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); - tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + + // Replace stream config from indexing config to ingestion config + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0) + .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); waitForAllDocsLoaded(600_000L); } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index bcd490572aaf..60e568acf5a4 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -70,6 +70,7 @@ import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; import org.apache.pinot.common.restlet.resources.ServerSegmentsReloadCheckResponse; +import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; import org.apache.pinot.common.restlet.resources.TableSegments; @@ -876,7 +877,7 @@ public String uploadLLCSegment( * @param segmentName name of the segment to be uploaded * @param timeoutMs timeout for the segment upload to the deep-store. If this is negative, the default timeout * would be used. - * @return full url where the segment is uploaded, crc, segmentName and other segment metadata. + * @return full url where the segment is uploaded, crc, segmentName. Can add more segment metadata in the future. * @throws Exception if an error occurred during the segment upload. */ @POST @@ -892,7 +893,106 @@ public String uploadLLCSegment( @ApiResponse(code = 404, message = "Table or segment not found", response = ErrorInfo.class), @ApiResponse(code = 400, message = "Bad request", response = ErrorInfo.class) }) - public String uploadLLCSegmentV2( + public TableLLCSegmentUploadResponse uploadLLCSegmentV2( + @ApiParam(value = "Name of the REALTIME table", required = true) @PathParam("realtimeTableNameWithType") + String realtimeTableNameWithType, + @ApiParam(value = "Name of the segment", required = true) @PathParam("segmentName") String segmentName, + @QueryParam("uploadTimeoutMs") @DefaultValue("-1") int timeoutMs, + @Context HttpHeaders headers) + throws Exception { + realtimeTableNameWithType = DatabaseUtils.translateTableName(realtimeTableNameWithType, headers); + LOGGER.info("Received a request to upload low level consumer segment {} for table {}", segmentName, + realtimeTableNameWithType); + + // Check it's realtime table + TableType tableType = TableNameBuilder.getTableTypeFromTableName(realtimeTableNameWithType); + if (TableType.REALTIME != tableType) { + throw new WebApplicationException( + String.format("Cannot upload low level consumer segment for a non-realtime table: %s", + realtimeTableNameWithType), Response.Status.BAD_REQUEST); + } + + // Check the segment is low level consumer segment + if (!LLCSegmentName.isLLCSegment(segmentName)) { + throw new WebApplicationException(String.format("Segment %s is not a low level consumer segment", segmentName), + Response.Status.BAD_REQUEST); + } + + TableDataManager tableDataManager = + ServerResourceUtils.checkGetTableDataManager(_serverInstance, realtimeTableNameWithType); + SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName); + if (segmentDataManager == null) { + throw new WebApplicationException( + String.format("Table %s segment %s does not exist", realtimeTableNameWithType, segmentName), + Response.Status.NOT_FOUND); + } + String crc = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); + + File segmentTarFile = null; + try { + // Create the tar.gz segment file in the server's segmentTarUploadDir folder with a unique file name. + File segmentTarUploadDir = + new File(_serverInstance.getInstanceDataManager().getSegmentFileDirectory(), SEGMENT_UPLOAD_DIR); + segmentTarUploadDir.mkdir(); + + segmentTarFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(segmentTarUploadDir, + realtimeTableNameWithType + "_" + segmentName + "_" + UUID.randomUUID() + + TarCompressionUtils.TAR_GZ_FILE_EXTENSION, "Invalid table / segment name: %s, %s", + realtimeTableNameWithType, segmentName); + + TarCompressionUtils.createCompressedTarFile(new File(tableDataManager.getTableDataDir(), segmentName), + segmentTarFile); + + // Use segment uploader to upload the segment tar file to segment store and return the segment download url. + SegmentUploader segmentUploader = _serverInstance.getInstanceDataManager().getSegmentUploader(); + URI segmentDownloadUrl; + if (timeoutMs <= 0) { + // Use default timeout if passed timeout is not positive + segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName)); + } else { + segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName), timeoutMs); + } + if (segmentDownloadUrl == null) { + throw new WebApplicationException( + String.format("Failed to upload table %s segment %s to segment store", realtimeTableNameWithType, + segmentName), Response.Status.INTERNAL_SERVER_ERROR); + } + return new TableLLCSegmentUploadResponse(segmentName, Long.parseLong(crc), segmentDownloadUrl.toString()); + } finally { + FileUtils.deleteQuietly(segmentTarFile); + tableDataManager.releaseSegment(segmentDataManager); + } + } + + /** + * Upload a low level consumer segment to segment store and return the segment download url, crc and + * other segment metadata. This endpoint is used when segment store copy is unavailable for committed + * low level consumer segments. + * Please note that invocation of this endpoint may cause query performance to suffer, since we tar up the segment + * to upload it. + * + * @see Date: Thu, 16 Jan 2025 00:22:41 +0530 Subject: [PATCH 22/65] Removing unnecessary code --- .../controller/validation/RealtimeSegmentValidationManager.java | 1 - .../realtime/FailureInjectingRealtimeSegmentDataManager.java | 1 - 2 files changed, 2 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 1d387223e3f1..95d348cce504 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -173,7 +173,6 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); -// _llcRealtimeSegmentManager.resetUploadedSegmentsInErrorState(tableConfig.getTableName()); // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java index 475533069494..dc6fce1915cf 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java @@ -59,7 +59,6 @@ public FailureInjectingRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMet protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { if (_failCommit) { - System.out.println("Forcing failure in buildSegmentInternal"); throw new RuntimeException("Forced failure in buildSegmentInternal"); } return super.buildSegmentInternal(forCommit); From ded8962eb978439405415030072d7f91a15408a3 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 16 Jan 2025 08:18:22 +0530 Subject: [PATCH 23/65] Fixing existing test cases and adding unit tests to check upload to deepstore path with fallbacks --- .../PinotLLCRealtimeSegmentManagerTest.java | 133 +++++++++++++++++- 1 file changed, 132 insertions(+), 1 deletion(-) diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 4980c6fe9aeb..8c9b3b97e360 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -51,6 +51,7 @@ import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.URIUtils; @@ -1114,6 +1115,135 @@ public void testUploadToSegmentStoreV2() segmentManager.createSegmentPath(RAW_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()).toString(); SegmentZKMetadata segmentZKMetadataCopy = new SegmentZKMetadata(new ZNRecord(segmentsZKMetadata.get(0).toZNRecord())); + + when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl0)).thenReturn( + new TableLLCSegmentUploadResponse(segmentsZKMetadata.get(0).getSegmentName(), 12345678L, + tempSegmentFileLocation.getPath())); + + // Change 2nd segment status to be DONE, but with default peer download url. + // Verify later the download url isn't fixed after upload failure. + segmentsZKMetadata.get(1).setStatus(Status.DONE); + segmentsZKMetadata.get(1).setDownloadUrl(METADATA_URI_FOR_PEER_DOWNLOAD); + // set up the external view for 2nd segment + String instance1 = "instance1"; + externalView.setState(segmentsZKMetadata.get(1).getSegmentName(), instance1, "ONLINE"); + InstanceConfig instanceConfig1 = new InstanceConfig(instance1); + instanceConfig1.setHostName(instance1); + instanceConfig1.getRecord().setIntField(Instance.ADMIN_PORT_KEY, adminPort); + when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance1)).thenReturn(instanceConfig1); + // mock the request/response for 2nd segment upload + String serverUploadRequestUrl1 = + String.format("http://%s:%d/segments/%s/%s/uploadLLCSegment?uploadTimeoutMs=-1", instance1, adminPort, + REALTIME_TABLE_NAME, segmentsZKMetadata.get(1).getSegmentName()); + when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl1)).thenThrow( + new HttpErrorStatusException("failed to upload segment", + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode())); + + // Change 3rd segment status to be DONE, but with default peer download url. + // Verify later the download url isn't fixed because no ONLINE replica found in any server. + segmentsZKMetadata.get(2).setStatus(Status.DONE); + segmentsZKMetadata.get(2).setDownloadUrl(METADATA_URI_FOR_PEER_DOWNLOAD); + // set up the external view for 3rd segment + String instance2 = "instance2"; + externalView.setState(segmentsZKMetadata.get(2).getSegmentName(), instance2, "OFFLINE"); + + // Change 4th segment status to be DONE and with segment download url. + // Verify later the download url is still the same. + String defaultDownloadUrl = "canItBeDownloaded"; + segmentsZKMetadata.get(3).setStatus(Status.DONE); + segmentsZKMetadata.get(3).setDownloadUrl(defaultDownloadUrl); + + // Keep 5th segment status as IN_PROGRESS. + + List segmentNames = + segmentsZKMetadata.stream().map(SegmentZKMetadata::getSegmentName).collect(Collectors.toList()); + when(pinotHelixResourceManager.getTableConfig(REALTIME_TABLE_NAME)).thenReturn(segmentManager._tableConfig); + + // Verify the result + segmentManager.uploadToDeepStoreIfMissing(segmentManager._tableConfig, segmentsZKMetadata); + + // Block until all tasks have been able to complete + TestUtils.waitForCondition(aVoid -> segmentManager.deepStoreUploadExecutorPendingSegmentsIsEmpty(), 30_000L, + "Timed out waiting for upload retry tasks to finish"); + + assertEquals(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(0), null).getDownloadUrl(), + expectedSegmentLocation); + assertFalse(tempSegmentFileLocation.exists(), + "Deep-store retry task should move the file from temp location to permanent location"); + + assertEquals(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(1), null).getDownloadUrl(), + METADATA_URI_FOR_PEER_DOWNLOAD); + assertEquals(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(2), null).getDownloadUrl(), + METADATA_URI_FOR_PEER_DOWNLOAD); + assertEquals(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(3), null).getDownloadUrl(), + defaultDownloadUrl); + assertNull(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(4), null).getDownloadUrl()); + } + + @Test + public void testUploadLLCSegmentToDeepStore() + throws HttpErrorStatusException, IOException, URISyntaxException { + // mock the behavior for PinotHelixResourceManager + PinotHelixResourceManager pinotHelixResourceManager = mock(PinotHelixResourceManager.class); + HelixManager helixManager = mock(HelixManager.class); + HelixAdmin helixAdmin = mock(HelixAdmin.class); + ZkHelixPropertyStore zkHelixPropertyStore = + (ZkHelixPropertyStore) mock(ZkHelixPropertyStore.class); + when(pinotHelixResourceManager.getHelixZkManager()).thenReturn(helixManager); + when(helixManager.getClusterManagmentTool()).thenReturn(helixAdmin); + when(helixManager.getClusterName()).thenReturn(CLUSTER_NAME); + when(pinotHelixResourceManager.getPropertyStore()).thenReturn(zkHelixPropertyStore); + + // init fake PinotLLCRealtimeSegmentManager + ControllerConf controllerConfig = new ControllerConf(); + controllerConfig.setProperty(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, + true); + controllerConfig.setDataDir(TEMP_DIR.toString()); + FakePinotLLCRealtimeSegmentManager segmentManager = + new FakePinotLLCRealtimeSegmentManager(pinotHelixResourceManager, controllerConfig); + Assert.assertTrue(segmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()); + + // Set up a new table with 2 replicas, 5 instances, 5 partition. + setUpNewTable(segmentManager, 2, 5, 5); + SegmentsValidationAndRetentionConfig segmentsValidationAndRetentionConfig = + new SegmentsValidationAndRetentionConfig(); + segmentsValidationAndRetentionConfig.setRetentionTimeUnit(TimeUnit.DAYS.toString()); + segmentsValidationAndRetentionConfig.setRetentionTimeValue("3"); + segmentManager._tableConfig.setValidationConfig(segmentsValidationAndRetentionConfig); + List segmentsZKMetadata = new ArrayList<>(segmentManager._segmentZKMetadataMap.values()); + Assert.assertEquals(segmentsZKMetadata.size(), 5); + + // Set up external view for this table + ExternalView externalView = new ExternalView(REALTIME_TABLE_NAME); + when(helixAdmin.getResourceExternalView(CLUSTER_NAME, REALTIME_TABLE_NAME)).thenReturn(externalView); + + // Change 1st segment status to be DONE, but with default peer download url. + // Verify later the download url is fixed after upload success. + segmentsZKMetadata.get(0).setStatus(Status.DONE); + segmentsZKMetadata.get(0).setDownloadUrl(METADATA_URI_FOR_PEER_DOWNLOAD); + // set up the external view for 1st segment + String instance0 = "instance0"; + int adminPort = 2077; + externalView.setState(segmentsZKMetadata.get(0).getSegmentName(), instance0, "ONLINE"); + InstanceConfig instanceConfig0 = new InstanceConfig(instance0); + instanceConfig0.setHostName(instance0); + instanceConfig0.getRecord().setIntField(Instance.ADMIN_PORT_KEY, adminPort); + when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance0)).thenReturn(instanceConfig0); + // mock the request/response for 1st segment upload + String serverUploadRequestUrl0 = + String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance0, adminPort, + REALTIME_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()); + // tempSegmentFileLocation is the location where the segment uploader will upload the segment. This usually ends + // with a random UUID + File tempSegmentFileLocation = new File(TEMP_DIR, segmentsZKMetadata.get(0).getSegmentName() + UUID.randomUUID()); + FileUtils.write(tempSegmentFileLocation, "test"); + // After the deep-store retry task gets the segment location returned by Pinot server, it will move the segment to + // its final location. This is the expected segment location. + String expectedSegmentLocation = + segmentManager.createSegmentPath(RAW_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()).toString(); + SegmentZKMetadata segmentZKMetadataCopy = + new SegmentZKMetadata(new ZNRecord(segmentsZKMetadata.get(0).toZNRecord())); + segmentZKMetadataCopy.setDownloadUrl(tempSegmentFileLocation.getPath()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( serverUploadRequestUrl0)).thenReturn( @@ -1132,7 +1262,7 @@ public void testUploadToSegmentStoreV2() when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance1)).thenReturn(instanceConfig1); // mock the request/response for 2nd segment upload String serverUploadRequestUrl1 = - String.format("http://%s:%d/segments/%s/%s/uploadLLCSegment?uploadTimeoutMs=-1", instance1, adminPort, + String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance1, adminPort, REALTIME_TABLE_NAME, segmentsZKMetadata.get(1).getSegmentName()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl1)).thenThrow( new HttpErrorStatusException("failed to upload segment", @@ -1179,6 +1309,7 @@ public void testUploadToSegmentStoreV2() assertNull(segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentNames.get(4), null).getDownloadUrl()); } + @Test public void testDeleteTmpSegmentFiles() throws Exception { From e703d84c1122e033bde5a8eb73128e16435218bb Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 16 Jan 2025 10:55:45 +0530 Subject: [PATCH 24/65] Refactor test cases to reduce repetition --- .../PinotLLCRealtimeSegmentManagerTest.java | 6 +- .../BasePauselessRealtimeIngestionTest.java | 231 ++++++++++++++++ ...IngestionCommitEndMetadataFailureTest.java | 218 ++------------- ...eIngestionIdealStateUpdateFailureTest.java | 251 ++---------------- ...elessRealtimeIngestionIntegrationTest.java | 159 ++--------- ...NewSegmentMetadataCreationFailureTest.java | 251 ++---------------- .../utils/PauselessRealtimeTestUtils.java | 91 +++++++ 7 files changed, 400 insertions(+), 807 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 8c9b3b97e360..84d70232ca7c 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -1231,7 +1231,8 @@ public void testUploadLLCSegmentToDeepStore() when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance0)).thenReturn(instanceConfig0); // mock the request/response for 1st segment upload String serverUploadRequestUrl0 = - String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance0, adminPort, + String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance0, + adminPort, REALTIME_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()); // tempSegmentFileLocation is the location where the segment uploader will upload the segment. This usually ends // with a random UUID @@ -1262,7 +1263,8 @@ public void testUploadLLCSegmentToDeepStore() when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance1)).thenReturn(instanceConfig1); // mock the request/response for 2nd segment upload String serverUploadRequestUrl1 = - String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance1, adminPort, + String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance1, + adminPort, REALTIME_TABLE_NAME, segmentsZKMetadata.get(1).getSegmentName()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl1)).thenThrow( new HttpErrorStatusException("failed to upload segment", diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java new file mode 100644 index 000000000000..96e1be8f9df1 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -0,0 +1,231 @@ +package org.apache.pinot.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.integration.tests.realtime.utils.PauselessRealtimeTestUtils; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertTrue; + + +public abstract class BasePauselessRealtimeIngestionTest extends BaseClusterIntegrationTest { + protected static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + protected static final long DEFAULT_COUNT_STAR_RESULT = 115545L; + protected static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + + protected List _avroFiles; + protected boolean _failureEnabled = false; + private static final Logger LOGGER = LoggerFactory.getLogger(BasePauselessRealtimeIngestionTest.class); + + protected abstract String getFailurePoint(); + + protected abstract int getExpectedSegmentsWithFailure(); + + protected abstract int getExpectedZKMetadataWithFailure(); + + protected abstract long getCountStarResultWithFailure(); + + @Override + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + serverConf.setProperty("pinot.server.instance.segment.store.uri", + "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + startZk(); + startController(); + startBroker(); + startServer(); + + setupNonPauselessTable(); + injectFailure(); + setupPauselessTable(); + waitForAllDocsLoaded(600_000L); + } + + private void setupNonPauselessTable() + throws Exception { + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig); + + waitForDocsLoaded(600_000L, true, tableConfig.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig.getTableName()); + return PauselessRealtimeTestUtils.assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + } + + private void setupPauselessTable() + throws Exception { + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0) + .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + + addTableConfig(tableConfig); + } + + protected void injectFailure() { + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .enableTestFault(getFailurePoint()); + _failureEnabled = true; + } + + protected void disableFailure() { + _failureEnabled = false; + _helixResourceManager.getPinotLLCRealtimeSegmentManager() + .disableTestFault(getFailurePoint()); + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + protected long getCountStarResult() { + return _failureEnabled ? getCountStarResultWithFailure() : DEFAULT_COUNT_STAR_RESULT; + } + + protected void runValidationAndVerify() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + String tableNameWithType2 = TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2); + + PauselessRealtimeTestUtils.verifyIdealState(tableNameWithType, getExpectedSegmentsWithFailure(), _helixManager); + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return segmentZKMetadataList.size() == getExpectedZKMetadataWithFailure(); + }, 1000, 100000, "New Segment ZK Metadata not created"); + + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + disableFailure(); + + _controllerStarter.getRealtimeSegmentValidationManager().run(); + + waitForAllDocsLoaded(600_000L); + waitForDocsLoaded(600_000L, true, tableNameWithType2); + + PauselessRealtimeTestUtils.verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS, _helixManager); + PauselessRealtimeTestUtils.verifyIdealState(tableNameWithType2, NUM_REALTIME_SEGMENTS, _helixManager); + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return PauselessRealtimeTestUtils.assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + PauselessRealtimeTestUtils.compareZKMetadataForSegments( + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); + } + + /** + * Basic test to verify segment assignment and metadata without any failures + */ + protected void testBasicSegmentAssignment() { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + + PauselessRealtimeTestUtils.verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS, _helixManager); + assertTrue(PauselessConsumptionUtils.isPauselessEnabled(getRealtimeTableConfig())); + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return !hasSegmentsInStatus(segmentZKMetadataList, CommonConstants.Segment.Realtime.Status.COMMITTING); + }, 1000, 100000, "Some segments have status COMMITTING"); + + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + return PauselessRealtimeTestUtils.assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + } + + private boolean hasSegmentsInStatus(List segmentZKMetadataList, + CommonConstants.Segment.Realtime.Status prohibitedStatus) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == prohibitedStatus) { + return true; + } + } + return false; + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java index 65d6d78bff14..0c816b656371 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionCommitEndMetadataFailureTest.java @@ -18,228 +18,50 @@ */ package org.apache.pinot.integration.tests; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import org.apache.commons.io.FileUtils; -import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; -import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.integration.tests.realtime.utils.PauselessRealtimeTestUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.util.TestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; -import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; -public class PauselessRealtimeIngestionCommitEndMetadataFailureTest extends BaseClusterIntegrationTest { +public class PauselessRealtimeIngestionCommitEndMetadataFailureTest + extends BasePauselessRealtimeIngestionTest { - private static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES - private static final Logger LOGGER = - LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); - private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; - private List _avroFiles; - - protected void overrideControllerConf(Map properties) { - properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); - properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); - properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", - "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); - // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. - // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. - properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, - 500); + @Override + protected String getFailurePoint() { + return FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA; } @Override - protected void overrideServerConf(PinotConfiguration serverConf) { - // Set segment store uri to the one used by controller as data dir (i.e. deep store) - try { - LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), - new URI(_controllerConfig.getDataDir()).getScheme()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); - serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, - "true"); + protected int getExpectedSegmentsWithFailure() { + return NUM_REALTIME_SEGMENTS; // All segments still appear in ideal state } - @BeforeClass - public void setUp() - throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - - // Start the Pinot cluster - startZk(); - // Start a customized controller with more frequent realtime segment validation - startController(); - startBroker(); - startServer(); - - // load data in kafka - _avroFiles = unpackAvroData(_tempDir); - startKafka(); - pushAvroIntoKafka(_avroFiles); - - // create schema for non-pauseless table - Schema schema = createSchema(); - schema.setSchemaName(DEFAULT_TABLE_NAME_2); - addSchema(schema); - - // add non-pauseless table - TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); - tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); - addTableConfig(tableConfig2); - - // Ensure that the commit protocol for all the segments have completed before injecting failure - waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = - _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - // inject failure in the commit protocol for the pauseless table - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA); - - // create schema for pauseless table - schema.setSchemaName(DEFAULT_TABLE_NAME); - addSchema(schema); - - // add pauseless table - TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - - // Replace stream config from indexing config to ingestion config - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); - ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); - ingestionConfig.getStreamIngestionConfig() - .getStreamConfigMaps() - .get(0) - .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); - tableConfig.getIndexingConfig().setStreamConfigs(null); - tableConfig.setIngestionConfig(ingestionConfig); + @Override + protected int getExpectedZKMetadataWithFailure() { + return NUM_REALTIME_SEGMENTS; + } - addTableConfig(tableConfig); - waitForAllDocsLoaded(600_000L); + @Override + protected long getCountStarResultWithFailure() { + return DEFAULT_COUNT_STAR_RESULT; } @Test public void testSegmentAssignment() throws Exception { String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); - assertUploadUrlEmpty(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType)); - // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can - // run segment level validations. The segment is not fixed by the validation manager in case the desired time - // can not elapsed - Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); - _controllerStarter.getRealtimeSegmentValidationManager().run(); - // wait for the url to show up after running validation manager - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); + PauselessRealtimeTestUtils.verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS, _helixManager); - compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), - _helixResourceManager.getSegmentsZKMetadata(TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); - } - - private void compareZKMetadataForSegments(List segmentsZKMetadata, - List segmentsZKMetadata1) { - Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); - Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); - segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { - SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); - areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); - }); - } - - private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { - if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { - return; - } - assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); - assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); - assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); - assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); - assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); - assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); - } - - private Map getPartitionSegmentNumberToMetadataMap( - List segmentsZKMetadata) { - Map segmentZKMetadataMap = new HashMap<>(); - for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); - String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); - segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); - } - return segmentZKMetadataMap; - } - - @AfterClass - public void tearDown() - throws IOException { - LOGGER.info("Tearing down..."); - dropRealtimeTable(getTableName()); - stopServer(); - stopBroker(); - stopController(); - stopKafka(); - stopZk(); - FileUtils.deleteDirectory(_tempDir); - } - - private void verifyIdealState(String tableName, int numSegmentsExpected) { - IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); - Map> segmentAssignment = idealState.getRecord().getMapFields(); - assertEquals(segmentAssignment.size(), numSegmentsExpected); - } - - private void assertUploadUrlEmpty(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - assertNull(segmentZKMetadata.getDownloadUrl()); + List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); + for (SegmentZKMetadata metadata : segmentZKMetadataList) { + assertNull(metadata.getDownloadUrl()); } - } - private boolean assertUrlPresent(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING - && segmentZKMetadata.getDownloadUrl() == null) { - LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); - return false; - } - } - return true; + runValidationAndVerify(); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java index 1d3d846c7bb7..a0ca117cf7b6 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIdealStateUpdateFailureTest.java @@ -18,259 +18,40 @@ */ package org.apache.pinot.integration.tests; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.io.FileUtils; -import org.apache.helix.model.IdealState; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; -import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.util.TestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; +public class PauselessRealtimeIngestionIdealStateUpdateFailureTest + extends BasePauselessRealtimeIngestionTest { -public class PauselessRealtimeIngestionIdealStateUpdateFailureTest extends BaseClusterIntegrationTest { - private static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES private static final int NUM_REALTIME_SEGMENTS_WITH_FAILURE = 2; private static final int NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE = 4; - protected static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; - private static final Logger LOGGER = - LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); - private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; - private List _avroFiles; - private static boolean _failureEnabled = false; + private static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; - protected void overrideControllerConf(Map properties) { - properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); - properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); - properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", - "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); - // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. - // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. - properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, - 500); + @Override + protected String getFailurePoint() { + return FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE; } @Override - protected void overrideServerConf(PinotConfiguration serverConf) { - // Set segment store uri to the one used by controller as data dir (i.e. deep store) - try { - LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), - new URI(_controllerConfig.getDataDir()).getScheme()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); - serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, - "true"); + protected int getExpectedSegmentsWithFailure() { + return NUM_REALTIME_SEGMENTS_WITH_FAILURE; } - @BeforeClass - public void setUp() - throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - - // Start the Pinot cluster - startZk(); - // Start a customized controller with more frequent realtime segment validation - startController(); - startBroker(); - startServer(); - - // load data in kafka - _avroFiles = unpackAvroData(_tempDir); - startKafka(); - pushAvroIntoKafka(_avroFiles); - - // create schema for non-pauseless table - Schema schema = createSchema(); - schema.setSchemaName(DEFAULT_TABLE_NAME_2); - addSchema(schema); - - // add non-pauseless table - TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); - tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); - addTableConfig(tableConfig2); - - // Ensure that the commit protocol for all the segments have completed before injecting failure - waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = - _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - // inject failure in the commit protocol for the pauseless table - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE); - _failureEnabled = true; - - // create schema for pauseless table - schema.setSchemaName(DEFAULT_TABLE_NAME); - addSchema(schema); - - // add pauseless table - TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - - // Replace stream config from indexing config to ingestion config - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); - ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); - ingestionConfig.getStreamIngestionConfig() - .getStreamConfigMaps() - .get(0) - .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); - tableConfig.getIndexingConfig().setStreamConfigs(null); - tableConfig.setIngestionConfig(ingestionConfig); + @Override + protected int getExpectedZKMetadataWithFailure() { + return NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; + } - addTableConfig(tableConfig); - waitForAllDocsLoaded(600_000L); + @Override + protected long getCountStarResultWithFailure() { + return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; } @Test public void testSegmentAssignment() throws Exception { - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); - String tableNameWithType2 = TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2); - // ensure that the metadata and ideal state only contain 2 segments. - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS_WITH_FAILURE); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return segmentZKMetadataList.size() == NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; - }, 1000, 100000, "New Segment ZK Metadata not created"); - - // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can - // run segment level validations. The segment is not fixed by the validation manager in case the desired time - // can not elapsed - Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); - _failureEnabled = false; - // inject failure in the commit protocol for the pauseless table - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .disableTestFault(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE); - - // Run validation manager. This should - // 1. Fix url for the segment that failed commit - // 2. Restart ingestion - _controllerStarter.getRealtimeSegmentValidationManager().run(); - // verify all the documents are loaded for both the tables - waitForAllDocsLoaded(600_000L); - waitForDocsLoaded(600_000L, true, tableNameWithType2); - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); - verifyIdealState(tableNameWithType2, NUM_REALTIME_SEGMENTS); - // wait for the url to show up after running validation manager - - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), - _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); - } - - private void compareZKMetadataForSegments(List segmentsZKMetadata, - List segmentsZKMetadata1) { - Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); - Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); - segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { - SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); - areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); - }); - } - - protected long getCountStarResult() { - if (_failureEnabled) { - return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; - } - return DEFAULT_COUNT_STAR_RESULT; - } - - private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { - if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { - return; - } - assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); - assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); - assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); - assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); - assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); - assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); - } - - private Map getPartitionSegmentNumberToMetadataMap( - List segmentsZKMetadata) { - Map segmentZKMetadataMap = new HashMap<>(); - for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); - String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); - segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); - } - return segmentZKMetadataMap; - } - - @AfterClass - public void tearDown() - throws IOException { - LOGGER.info("Tearing down..."); - dropRealtimeTable(getTableName()); - stopServer(); - stopBroker(); - stopController(); - stopKafka(); - stopZk(); - FileUtils.deleteDirectory(_tempDir); - } - - private void verifyIdealState(String tableName, int numSegmentsExpected) { - IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); - Map> segmentAssignment = idealState.getRecord().getMapFields(); - assertEquals(segmentAssignment.size(), numSegmentsExpected); - } - - private void assertUploadUrlEmpty(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - assertNull(segmentZKMetadata.getDownloadUrl()); - } - } - - private boolean assertUrlPresent(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING - && segmentZKMetadata.getDownloadUrl() == null) { - LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); - return false; - } - } - return true; + runValidationAndVerify(); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java index 4e9fcac0abdc..bce6e67e3536 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionIntegrationTest.java @@ -18,159 +18,44 @@ */ package org.apache.pinot.integration.tests; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.List; -import java.util.Map; -import org.apache.commons.io.FileUtils; -import org.apache.helix.model.IdealState; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.PauselessConsumptionUtils; -import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; -import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.util.TestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; - -public class PauselessRealtimeIngestionIntegrationTest extends BaseClusterIntegrationTest { - - private static final int NUM_REALTIME_SEGMENTS = 48; - private static final Logger LOGGER = LoggerFactory.getLogger(PauselessRealtimeIngestionIntegrationTest.class); - private List _avroFiles; - - protected void overrideControllerConf(Map properties) { - properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); - properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); - properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", - "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); - } +public class PauselessRealtimeIngestionIntegrationTest extends BasePauselessRealtimeIngestionTest { @Override - protected void overrideServerConf(PinotConfiguration serverConf) { - // Set segment store uri to the one used by controller as data dir (i.e. deep store) - try { - LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), - new URI(_controllerConfig.getDataDir()).getScheme()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); - serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, - "true"); - } - - @BeforeClass - public void setUp() - throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - - // Start the Pinot cluster - startZk(); - // Start a customized controller with more frequent realtime segment validation - startController(); - startBroker(); - startServer(); - - _avroFiles = unpackAvroData(_tempDir); - startKafka(); - pushAvroIntoKafka(_avroFiles); - - Schema schema = createSchema(); - addSchema(schema); - TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); - // Replace stream config from indexing config to ingestion config - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); - ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); - tableConfig.getIndexingConfig().setStreamConfigs(null); - tableConfig.setIngestionConfig(ingestionConfig); - addTableConfig(tableConfig); - - waitForAllDocsLoaded(600_000L); + protected String getFailurePoint() { + return null; // No failure point for basic test } - @Test(description = "Ensure that all the segments are ingested, built and uploaded when pauseless consumption is " - + "enabled") - public void testSegmentAssignment() - throws Exception { - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); - assertTrue(PauselessConsumptionUtils.isPauselessEnabled(getRealtimeTableConfig())); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertNoSegmentInProhibitedStatus(segmentZKMetadataList, - CommonConstants.Segment.Realtime.Status.COMMITTING); - }, 1000, 100000, "Some segments have status COMMITTING"); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); + @Override + protected int getExpectedSegmentsWithFailure() { + return NUM_REALTIME_SEGMENTS; // Always expect full segments } - @AfterClass - public void tearDown() - throws IOException { - LOGGER.info("Tearing down..."); - dropRealtimeTable(getTableName()); - stopServer(); - stopBroker(); - stopController(); - stopKafka(); - stopZk(); - FileUtils.deleteDirectory(_tempDir); + @Override + protected int getExpectedZKMetadataWithFailure() { + return NUM_REALTIME_SEGMENTS; // Always expect full metadata } - private void verifyIdealState(String tableName, int numSegmentsExpected) { - IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); - Map> segmentAssignment = idealState.getRecord().getMapFields(); - assertEquals(segmentAssignment.size(), numSegmentsExpected); + @Override + protected long getCountStarResultWithFailure() { + return DEFAULT_COUNT_STAR_RESULT; // Always expect full count } - private boolean assertUrlPresent(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE - && segmentZKMetadata.getDownloadUrl() == null) { - System.out.println("URl not found for segment: " + segmentZKMetadata.getSegmentName()); - return false; - } - } - return true; + @Override + protected void injectFailure() { + // Do nothing - no failure to inject } - private boolean assertNoSegmentInProhibitedStatus(List segmentZKMetadataList, - CommonConstants.Segment.Realtime.Status prohibitedStatus) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == prohibitedStatus) { - return false; - } - } - return true; + @Override + protected void disableFailure() { + // Do nothing - no failure to disable } - @Override - protected Map getStreamConfigs() { - Map streamConfigMap = getStreamConfigMap(); - streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); - return streamConfigMap; + @Test(description = "Ensure that all the segments are ingested, built and uploaded when pauseless consumption is " + + "enabled") + public void testSegmentAssignment() { + testBasicSegmentAssignment(); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java index 34b2fbb4b9d7..7af627ad7015 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest.java @@ -18,259 +18,40 @@ */ package org.apache.pinot.integration.tests; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.io.FileUtils; -import org.apache.helix.model.IdealState; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; -import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.util.TestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; +public class PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest + extends BasePauselessRealtimeIngestionTest { -public class PauselessRealtimeIngestionNewSegmentMetadataCreationFailureTest extends BaseClusterIntegrationTest { - private static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES private static final int NUM_REALTIME_SEGMENTS_WITH_FAILURE = 2; private static final int NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE = 2; - protected static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; - private static final Logger LOGGER = - LoggerFactory.getLogger(PauselessRealtimeIngestionCommitEndMetadataFailureTest.class); - private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; - private List _avroFiles; - private static boolean _failureEnabled = false; + private static final long DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE = 5000; - protected void overrideControllerConf(Map properties) { - properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); - properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); - properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", - "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); - // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. - // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. - properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, - 500); + @Override + protected String getFailurePoint() { + return FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION; } @Override - protected void overrideServerConf(PinotConfiguration serverConf) { - // Set segment store uri to the one used by controller as data dir (i.e. deep store) - try { - LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), - new URI(_controllerConfig.getDataDir()).getScheme()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); - serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, - "true"); + protected int getExpectedSegmentsWithFailure() { + return NUM_REALTIME_SEGMENTS_WITH_FAILURE; } - @BeforeClass - public void setUp() - throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - - // Start the Pinot cluster - startZk(); - // Start a customized controller with more frequent realtime segment validation - startController(); - startBroker(); - startServer(); - - // load data in kafka - _avroFiles = unpackAvroData(_tempDir); - startKafka(); - pushAvroIntoKafka(_avroFiles); - - // create schema for non-pauseless table - Schema schema = createSchema(); - schema.setSchemaName(DEFAULT_TABLE_NAME_2); - addSchema(schema); - - // add non-pauseless table - TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); - tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); - addTableConfig(tableConfig2); - - // Ensure that the commit protocol for all the segments have completed before injecting failure - waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = - _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - // inject failure in the commit protocol for the pauseless table - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .enableTestFault(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION); - _failureEnabled = true; - - // create schema for pauseless table - schema.setSchemaName(DEFAULT_TABLE_NAME); - addSchema(schema); - - // add pauseless table - TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - - // Replace stream config from indexing config to ingestion config - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); - ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); - ingestionConfig.getStreamIngestionConfig() - .getStreamConfigMaps() - .get(0) - .put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); - tableConfig.getIndexingConfig().setStreamConfigs(null); - tableConfig.setIngestionConfig(ingestionConfig); + @Override + protected int getExpectedZKMetadataWithFailure() { + return NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; + } - addTableConfig(tableConfig); - waitForAllDocsLoaded(600_000L); + @Override + protected long getCountStarResultWithFailure() { + return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; } @Test public void testSegmentAssignment() throws Exception { - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); - String tableNameWithType2 = TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2); - // ensure that the metadata and ideal state only contain 2 segments. - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS_WITH_FAILURE); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return segmentZKMetadataList.size() == NUM_REALTIME_SEGMENTS_ZK_METADATA_WITH_FAILURE; - }, 1000, 100000, "New Segment ZK Metadata not created"); - - // this sleep has been introduced to ensure that the RealtimeSegmentValidationManager can - // run segment level validations. The segment is not fixed by the validation manager in case the desired time - // can not elapsed - Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); - _failureEnabled = false; - // inject failure in the commit protocol for the pauseless table - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .disableTestFault(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION); - - // Run validation manager. This should - // 1. Fix url for the segment that failed commit - // 2. Restart ingestion - _controllerStarter.getRealtimeSegmentValidationManager().run(); - // verify all the documents are loaded for both the tables - waitForAllDocsLoaded(600_000L); - waitForDocsLoaded(600_000L, true, tableNameWithType2); - verifyIdealState(tableNameWithType, NUM_REALTIME_SEGMENTS); - verifyIdealState(tableNameWithType2, NUM_REALTIME_SEGMENTS); - // wait for the url to show up after running validation manager - - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = _helixResourceManager.getSegmentsZKMetadata(tableNameWithType); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), - _helixResourceManager.getSegmentsZKMetadata(tableNameWithType2)); - } - - private void compareZKMetadataForSegments(List segmentsZKMetadata, - List segmentsZKMetadata1) { - Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); - Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); - segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { - SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); - areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); - }); - } - - protected long getCountStarResult() { - if (_failureEnabled) { - return DEFAULT_COUNT_STAR_RESULT_WITH_FAILURE; - } - return DEFAULT_COUNT_STAR_RESULT; - } - - private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { - if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { - return; - } - assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); - assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); - assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); - assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); - assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); - assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); - } - - private Map getPartitionSegmentNumberToMetadataMap( - List segmentsZKMetadata) { - Map segmentZKMetadataMap = new HashMap<>(); - for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); - String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); - segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); - } - return segmentZKMetadataMap; - } - - @AfterClass - public void tearDown() - throws IOException { - LOGGER.info("Tearing down..."); - dropRealtimeTable(getTableName()); - stopServer(); - stopBroker(); - stopController(); - stopKafka(); - stopZk(); - FileUtils.deleteDirectory(_tempDir); - } - - private void verifyIdealState(String tableName, int numSegmentsExpected) { - IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); - Map> segmentAssignment = idealState.getRecord().getMapFields(); - assertEquals(segmentAssignment.size(), numSegmentsExpected); - } - - private void assertUploadUrlEmpty(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - assertNull(segmentZKMetadata.getDownloadUrl()); - } - } - - private boolean assertUrlPresent(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING - && segmentZKMetadata.getDownloadUrl() == null) { - LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); - return false; - } - } - return true; + runValidationAndVerify(); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java new file mode 100644 index 000000000000..4196b43dcfd7 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java @@ -0,0 +1,91 @@ +/** + * 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.integration.tests.realtime.utils; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.helix.HelixManager; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.spi.utils.CommonConstants; + +import static org.testng.Assert.assertEquals; + + +public class PauselessRealtimeTestUtils { + + private PauselessRealtimeTestUtils() { + } + + public static void verifyIdealState(String tableName, int numSegmentsExpected, HelixManager helixManager) { + IdealState idealState = HelixHelper.getTableIdealState(helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + public static boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + return false; + } + } + return true; + } + + public static void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = + getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = + getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + compareSegmentZkMetadata(segmentZKMetadata, segmentZKMetadata1); + }); + } + + private static Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + private static void compareSegmentZkMetadata(SegmentZKMetadata segmentZKMetadata, + SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } +} From c2fda4a2d3a01fb43c136c95775ee0c05b239363 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 16 Jan 2025 10:58:34 +0530 Subject: [PATCH 25/65] Add missing header file --- .../BasePauselessRealtimeIngestionTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index 96e1be8f9df1..38ad2f7e47cc 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -1,3 +1,21 @@ +/** + * 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.integration.tests; import java.io.File; From c836009e15a5c61b9ec8594e0b478da9c979ffa5 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Fri, 17 Jan 2025 10:49:32 +0530 Subject: [PATCH 26/65] Fix reingestion test --- .../PinotLLCRealtimeSegmentManager.java | 5 +++++ .../RealtimeSegmentValidationManager.java | 9 ++++++++- ...altimeIngestionSegmentCommitFailureTest.java | 17 +++++++++++++++-- ...lureInjectingRealtimeSegmentDataManager.java | 4 ++-- 4 files changed, 30 insertions(+), 5 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 51b2f0c17e32..9264eed06a7f 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2237,6 +2237,11 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { } } + if (segmentsInErrorState.isEmpty()) { + LOGGER.info("No segments found in ERROR state for table {}", tableNameWithType); + return; + } + // filter out segments that are not ONLINE in IdealState for (String segmentName : segmentsInErrorState) { Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 95d348cce504..e4411f7ae441 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -28,6 +28,7 @@ import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.metrics.ValidationMetrics; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.LeadControllerManager; import org.apache.pinot.controller.api.resources.PauseStatusDetails; @@ -115,6 +116,8 @@ protected void processTable(String tableNameWithType, Context context) { if (context._runSegmentLevelValidation) { runSegmentLevelValidation(tableConfig); + } else { + LOGGER.info("Skipping segment-level validation for table: {}", tableConfig.getTableName()); } } @@ -172,7 +175,11 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { // Update the total document count gauge _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); - _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); + boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); + + if (isPauselessConsumptionEnabled) { + _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); + } // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java index fb23f22a50e4..e37714d9976c 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -38,6 +38,8 @@ import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; @@ -136,8 +138,19 @@ public void setUp() TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - tableConfig.getIndexingConfig().setPauselessConsumptionEnabled(true); - tableConfig.getIndexingConfig().getStreamConfigs().put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + Map streamConfigMap = ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0); + streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + streamConfigMap.put("segmentDownloadTimeoutMinutes", "1"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); Thread.sleep(60000L); TestUtils.waitForCondition( diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java index dc6fce1915cf..072cc92e4923 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java @@ -59,8 +59,8 @@ public FailureInjectingRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMet protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { if (_failCommit) { - throw new RuntimeException("Forced failure in buildSegmentInternal"); + throw new RuntimeException("Forced failure in buildSegmentInternal"); } - return super.buildSegmentInternal(forCommit); + return super.buildSegmentInternal(forCommit); } } From 7974ab926d6305f6cee721ec62db6cfc5263a627 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Sat, 18 Jan 2025 00:38:25 +0530 Subject: [PATCH 27/65] refactoring file upload download client --- .../common/utils/FileUploadDownloadClient.java | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index ca864a0a04f0..8147ef87f132 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -1301,25 +1301,13 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, URI reIngestUri = getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); - // Build the JSON payload Map requestJson = new HashMap<>(); requestJson.put("tableNameWithType", tableNameWithType); requestJson.put("segmentName", segmentName); - // Convert the request payload to JSON string String jsonPayload = JsonUtils.objectToString(requestJson); - // Prepare a POST request with Simple HTTP - ClassicRequestBuilder requestBuilder = ClassicRequestBuilder - .post(reIngestUri) - .setVersion(HttpVersion.HTTP_1_1) - .setHeader("Content-Type", "application/json") - .setHeader("Accept", "application/json") - .setEntity(new StringEntity(jsonPayload, ContentType.APPLICATION_JSON)); - - // Send the request using your custom HttpClient wrapper. - // (Adjust the timeout as needed in your environment) - SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( - _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); + SimpleHttpResponse response = + HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); // Check that we got a 2xx response int statusCode = response.getStatusCode(); From c08f84199c77cb49cc093316c4b774b0ffedbf92 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Tue, 21 Jan 2025 22:26:38 +0530 Subject: [PATCH 28/65] Removing pauselessConsumptionEnabled from index config --- .../apache/pinot/spi/config/table/IndexingConfig.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index cfe002480f15..5beb126e0d0e 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -66,7 +66,6 @@ public class IndexingConfig extends BaseJsonConfig { private boolean _aggregateMetrics; private boolean _nullHandlingEnabled; private boolean _columnMajorSegmentBuilderEnabled = true; - private boolean _pauselessConsumptionEnabled = false; /** * If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality @@ -460,12 +459,4 @@ public Set getAllReferencedColumns() { } return allColumns; } - - public boolean isPauselessConsumptionEnabled() { - return _pauselessConsumptionEnabled; - } - - public void setPauselessConsumptionEnabled(boolean pauselessConsumptionEnabled) { - _pauselessConsumptionEnabled = pauselessConsumptionEnabled; - } } From c4b99bd85a2d5a934656451a5c00364a6fdad0a1 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 21 Jan 2025 23:06:04 +0530 Subject: [PATCH 29/65] Remove reingestion code --- .../utils/FileUploadDownloadClient.java | 42 -- .../PinotLLCRealtimeSegmentManager.java | 136 ----- .../RealtimeSegmentValidationManager.java | 7 - ...timeIngestionSegmentCommitFailureTest.java | 312 ---------- .../api/resources/ReIngestionResource.java | 477 --------------- .../reingestion/ReIngestionRequest.java | 40 -- .../reingestion/ReIngestionResponse.java | 36 -- .../SimpleRealtimeSegmentDataManager.java | 569 ------------------ 8 files changed, 1619 deletions(-) delete mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java delete mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java delete mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java delete mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java delete mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 8147ef87f132..8bf5e576c005 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -127,7 +127,6 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; - public static final String REINGEST_SEGMENT_PATH = "/reingestSegment"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); @@ -1276,47 +1275,6 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi httpHeaders, maxStreamRateInByte); } - /** - * Invokes the server's reIngestSegment API via a POST request with JSON payload, - * using Simple HTTP APIs. - * - * POST http://[serverURL]/reIngestSegment - * { - * "tableNameWithType": [tableName], - * "segmentName": [segmentName] - * } - */ - public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName) - throws IOException, URISyntaxException, HttpErrorStatusException { - String scheme = HTTP; - if (serverHostPort.contains(HTTPS)) { - scheme = HTTPS; - serverHostPort = serverHostPort.replace(HTTPS + "://", ""); - } else if (serverHostPort.contains(HTTP)) { - serverHostPort = serverHostPort.replace(HTTP + "://", ""); - } - - String serverHost = serverHostPort.split(":")[0]; - String serverPort = serverHostPort.split(":")[1]; - - URI reIngestUri = getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); - - Map requestJson = new HashMap<>(); - requestJson.put("tableNameWithType", tableNameWithType); - requestJson.put("segmentName", segmentName); - - String jsonPayload = JsonUtils.objectToString(requestJson); - SimpleHttpResponse response = - HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); - - // Check that we got a 2xx response - int statusCode = response.getStatusCode(); - if (statusCode / 100 != 2) { - throw new IOException(String.format("Failed POST to %s, HTTP %d: %s", - reIngestUri, statusCode, response.getResponse())); - } - } - /** * Generate a param list with a table name attribute. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 9264eed06a7f..4710a2c8aa8b 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.collect.BiMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.io.IOException; @@ -49,10 +48,8 @@ import org.apache.helix.ClusterMessagingService; import org.apache.helix.Criteria; import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; -import org.apache.helix.PropertyKey; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.store.zk.ZkHelixPropertyStore; @@ -2196,139 +2193,6 @@ URI createSegmentPath(String rawTableName, String segmentName) { return URIUtils.getUri(_controllerConf.getDataDir(), rawTableName, URIUtils.encode(segmentName)); } - /** - * Re-ingests segments that are in DONE status with a missing download URL, but also - * have no peer copy on any server. This method will call the server reIngestSegment API - * on one of the alive servers that are supposed to host that segment according to IdealState. - * - * API signature: - * POST http://[serverURL]/reIngestSegment - * Request body (JSON): - * { - * "tableNameWithType": [tableName], - * "segmentName": [segmentName], - * "uploadURI": [leadControllerUrl], - * "uploadSegment": true - * } - * - * @param tableNameWithType The table name with type, e.g. "myTable_REALTIME" - */ - public void reIngestSegmentsWithErrorState(String tableNameWithType) { - // Step 1: Fetch the ExternalView and all segments - ExternalView externalView = getExternalView(tableNameWithType); - IdealState idealState = getIdealState(tableNameWithType); - Map> segmentToInstanceCurrentStateMap = externalView.getRecord().getMapFields(); - Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); - - // find segments in ERROR state in externalView - List segmentsInErrorState = new ArrayList<>(); - for (Map.Entry> entry : segmentToInstanceCurrentStateMap.entrySet()) { - String segmentName = entry.getKey(); - Map instanceStateMap = entry.getValue(); - boolean allReplicasInError = true; - for (String state : instanceStateMap.values()) { - if (!SegmentStateModel.ERROR.equals(state)) { - allReplicasInError = false; - break; - } - } - if (allReplicasInError) { - segmentsInErrorState.add(segmentName); - } - } - - if (segmentsInErrorState.isEmpty()) { - LOGGER.info("No segments found in ERROR state for table {}", tableNameWithType); - return; - } - - // filter out segments that are not ONLINE in IdealState - for (String segmentName : segmentsInErrorState) { - Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); - boolean isOnline = true; - for (String state : instanceIdealStateMap.values()) { - if (!SegmentStateModel.ONLINE.equals(state)) { - isOnline = false; - break; - } - } - if (!isOnline) { - segmentsInErrorState.remove(segmentName); - } - } - - // Step 2: For each segment, check the ZK metadata for conditions - for (String segmentName : segmentsInErrorState) { - // Skip non-LLC segments or segments missing from the ideal state altogether - LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); - if (llcSegmentName == null || !segmentToInstanceCurrentStateMap.containsKey(segmentName)) { - continue; - } - - SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); - // We only consider segments that are in COMMITTING which is indicated by having an endOffset - // but have a missing or placeholder download URL - if (segmentZKMetadata.getStatus() == Status.COMMITTING) { - Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); - - // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive - LOGGER.info( - "Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", - segmentName, tableNameWithType); - - // Find at least one server that should host this segment and is alive - String aliveServer = findAliveServerToReIngest(instanceStateMap.keySet()); - if (aliveServer == null) { - LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, tableNameWithType); - continue; - } - - try { - _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName); - LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); - } catch (Exception e) { - LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); - } - } else if (segmentZKMetadata.getStatus() == Status.UPLOADED) { - LOGGER.info( - "Segment {} in table {} is in ERROR state with download URL present. Resetting segment to ONLINE state.", - segmentName, tableNameWithType); - _helixResourceManager.resetSegment(tableNameWithType, segmentName, null); - } - } - } - - /** - * Picks one 'alive' server among a set of servers that are supposed to host the segment, - * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. - * This is a simple example; adapt to your environment’s definition of “alive.” - */ - private String findAliveServerToReIngest(Set candidateServers) { - // Get the current live instances from Helix - HelixDataAccessor helixDataAccessor = _helixManager.getHelixDataAccessor(); - PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder(); - List liveInstances = helixDataAccessor.getChildNames(keyBuilder.liveInstances()); - try { - // This should ideally handle https scheme as well - BiMap instanceToEndpointMap = - _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); - - if (instanceToEndpointMap.isEmpty()) { - LOGGER.warn("No instance data admin endpoints found for servers: {}", candidateServers); - return null; - } - - for (String server : candidateServers) { - if (liveInstances.contains(server)) { - return instanceToEndpointMap.get(server); - } - } - } catch (Exception e) { - LOGGER.warn("Failed to get Helix instance data admin endpoints for servers: {}", candidateServers, e); - } - return null; - } - @VisibleForTesting public void enableTestFault(String faultType) { _failureConfig.put(faultType, "true"); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index e4411f7ae441..510b292a83bb 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -28,7 +28,6 @@ import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.metrics.ValidationMetrics; -import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.LeadControllerManager; import org.apache.pinot.controller.api.resources.PauseStatusDetails; @@ -175,12 +174,6 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { // Update the total document count gauge _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); - boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); - - if (isPauselessConsumptionEnabled) { - _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); - } - // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java deleted file mode 100644 index e37714d9976c..000000000000 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java +++ /dev/null @@ -1,312 +0,0 @@ -/** - * 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.integration.tests; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.io.FileUtils; -import org.apache.helix.model.ExternalView; -import org.apache.helix.model.IdealState; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; -import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; -import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.util.TestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNull; - - -public class PauselessRealtimeIngestionSegmentCommitFailureTest extends BaseClusterIntegrationTest { - - private static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES - private static final Logger LOGGER = - LoggerFactory.getLogger(PauselessRealtimeIngestionSegmentCommitFailureTest.class); - private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; - private List _avroFiles; - - protected void overrideControllerConf(Map properties) { - properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); - properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); - properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", - "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); - // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. - // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. - properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, - 500); - } - - @Override - protected void overrideServerConf(PinotConfiguration serverConf) { - // Set segment store uri to the one used by controller as data dir (i.e. deep store) - try { - LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), - new URI(_controllerConfig.getDataDir()).getScheme()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); - serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, - "true"); - serverConf.setProperty("pinot.server.instance." + CommonConstants.Server.TABLE_DATA_MANAGER_PROVIDER_CLASS, - "org.apache.pinot.integration.tests.realtime.FailureInjectingTableDataManagerProvider"); - } - - @BeforeClass - public void setUp() - throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - - // Start the Pinot cluster - startZk(); - // Start a customized controller with more frequent realtime segment validation - startController(); - startBroker(); - startServer(); - - // load data in kafka - _avroFiles = unpackAvroData(_tempDir); - startKafka(); - pushAvroIntoKafka(_avroFiles); - - // create schema for non-pauseless table - Schema schema = createSchema(); - schema.setSchemaName(DEFAULT_TABLE_NAME_2); - addSchema(schema); - - // add non-pauseless table - TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); - tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); - addTableConfig(tableConfig2); - - // Ensure that the commit protocol for all the segments have completed before injecting failure - waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); - TestUtils.waitForCondition((aVoid) -> { - List segmentZKMetadataList = - _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); - return assertUrlPresent(segmentZKMetadataList); - }, 1000, 100000, "Some segments still have missing url"); - - // create schema for pauseless table - schema.setSchemaName(DEFAULT_TABLE_NAME); - addSchema(schema); - - // add pauseless table - TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); - tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); - tableConfig.getValidationConfig().setRetentionTimeValue("100000"); - - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setStreamIngestionConfig( - new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); - ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); - Map streamConfigMap = ingestionConfig.getStreamIngestionConfig() - .getStreamConfigMaps() - .get(0); - streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); - streamConfigMap.put("segmentDownloadTimeoutMinutes", "1"); - tableConfig.getIndexingConfig().setStreamConfigs(null); - tableConfig.setIngestionConfig(ingestionConfig); - - addTableConfig(tableConfig); - Thread.sleep(60000L); - TestUtils.waitForCondition( - (aVoid) -> atLeastOneErrorSegmentInExternalView(TableNameBuilder.REALTIME.tableNameWithType(getTableName())), - 1000, 600000, "Segments still not in error state"); - } - - @Test - public void testSegmentAssignment() - throws Exception { - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); - - // 1) Capture which segments went into the ERROR state - List erroredSegments = getErroredSegmentsInExternalView(tableNameWithType); - assertFalse(erroredSegments.isEmpty(), "No segments found in ERROR state, expected at least one."); - - // Let the RealtimeSegmentValidationManager run so it can fix up segments - Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); - LOGGER.info("Triggering RealtimeSegmentValidationManager to reingest errored segments"); - _controllerStarter.getRealtimeSegmentValidationManager().run(); - LOGGER.info("Finished RealtimeSegmentValidationManager to reingest errored segments"); - - // Wait until there are no ERROR segments in the ExternalView - TestUtils.waitForCondition(aVoid -> { - List errorSegmentsRemaining = getErroredSegmentsInExternalView(tableNameWithType); - return errorSegmentsRemaining.isEmpty(); - }, 10000, 100000, "Some segments are still in ERROR state after resetSegments()"); - - // Finally compare metadata across your two tables - compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), - _helixResourceManager.getSegmentsZKMetadata( - TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); - } - - /** - * Returns the list of segment names in ERROR state from the ExternalView of the given table. - */ - private List getErroredSegmentsInExternalView(String tableName) { - ExternalView resourceEV = _helixResourceManager.getHelixAdmin() - .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); - Map> segmentAssignment = resourceEV.getRecord().getMapFields(); - List erroredSegments = new ArrayList<>(); - for (Map.Entry> entry : segmentAssignment.entrySet()) { - String segmentName = entry.getKey(); - Map serverToStateMap = entry.getValue(); - for (String state : serverToStateMap.values()) { - if ("ERROR".equals(state)) { - erroredSegments.add(segmentName); - break; // No need to check other servers for this segment - } - } - } - return erroredSegments; - } - - /** - * Checks that all segments which were previously in ERROR state now have status == UPLOADED. - */ - private boolean haveErroredSegmentsUploaded( - List segmentZKMetadataList, List previouslyErroredSegments) { - - // Convert to a Set for quick lookups - Set erroredSegmentNames = new HashSet<>(previouslyErroredSegments); - - for (SegmentZKMetadata metadata : segmentZKMetadataList) { - if (erroredSegmentNames.contains(metadata.getSegmentName())) { - // If it was previously ERROR, then we expect it to have transitioned to UPLOADED - if (metadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { - return false; - } - } - } - return true; - } - - - private void compareZKMetadataForSegments(List segmentsZKMetadata, - List segmentsZKMetadata1) { - Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); - Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); - segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { - SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); - areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); - }); - } - - private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { - if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { - return; - } - assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); - assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); - assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); - assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); - assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); - assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); - } - - private Map getPartitionSegmentNumberToMetadataMap( - List segmentsZKMetadata) { - Map segmentZKMetadataMap = new HashMap<>(); - for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); - String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); - segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); - } - return segmentZKMetadataMap; - } - - @AfterClass - public void tearDown() - throws IOException { - LOGGER.info("Tearing down..."); - dropRealtimeTable(getTableName()); - stopServer(); - stopBroker(); - stopController(); - stopKafka(); - stopZk(); - FileUtils.deleteDirectory(_tempDir); - } - - private void verifyIdealState(String tableName, int numSegmentsExpected) { - IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); - Map> segmentAssignment = idealState.getRecord().getMapFields(); - assertEquals(segmentAssignment.size(), numSegmentsExpected); - } - - private boolean atLeastOneErrorSegmentInExternalView(String tableName) { - ExternalView resourceEV = _helixResourceManager.getHelixAdmin() - .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); - Map> segmentAssigment = resourceEV.getRecord().getMapFields(); - for (Map serverToStateMap : segmentAssigment.values()) { - for (String state : serverToStateMap.values()) { - if (state.equals("ERROR")) { - return true; - } - } - } - return false; - } - - private void assertUploadUrlEmpty(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - assertNull(segmentZKMetadata.getDownloadUrl()); - } - } - - private boolean assertUrlPresent(List segmentZKMetadataList) { - for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { - if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING - && segmentZKMetadata.getDownloadUrl() == null) { - LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); - return false; - } - } - return true; - } -} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java deleted file mode 100644 index 5c145beedcc0..000000000000 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ /dev/null @@ -1,477 +0,0 @@ -/** - * 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.server.api.resources; - -import com.google.common.base.Function; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiKeyAuthDefinition; -import io.swagger.annotations.ApiOperation; -import io.swagger.annotations.ApiResponse; -import io.swagger.annotations.ApiResponses; -import io.swagger.annotations.Authorization; -import io.swagger.annotations.SecurityDefinition; -import io.swagger.annotations.SwaggerDefinition; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import javax.inject.Inject; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.HttpHeaders; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.hc.core5.http.Header; -import org.apache.hc.core5.http.NameValuePair; -import org.apache.hc.core5.http.message.BasicHeader; -import org.apache.hc.core5.http.message.BasicNameValuePair; -import org.apache.pinot.common.auth.AuthProviderUtils; -import org.apache.pinot.common.exception.HttpErrorStatusException; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.FileUploadDownloadClient; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.SimpleHttpResponse; -import org.apache.pinot.common.utils.TarCompressionUtils; -import org.apache.pinot.common.utils.URIUtils; -import org.apache.pinot.common.utils.http.HttpClient; -import org.apache.pinot.core.data.manager.InstanceDataManager; -import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; -import org.apache.pinot.segment.local.data.manager.SegmentDataManager; -import org.apache.pinot.segment.local.data.manager.TableDataManager; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.segment.spi.V1Constants; -import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; -import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; -import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; -import org.apache.pinot.server.realtime.ControllerLeaderLocator; -import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; -import org.apache.pinot.server.starter.ServerInstance; -import org.apache.pinot.spi.auth.AuthProvider; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.TableType; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.IngestionConfigUtils; -import org.apache.pinot.spi.utils.StringUtil; -import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; -import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; -import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; - - -@Api(tags = "ReIngestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), - @Authorization(value = DATABASE)}) -@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = { - @ApiKeyAuthDefinition(name = HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, - key = SWAGGER_AUTHORIZATION_KEY, - description = "The format of the key is ```\"Basic \" or \"Bearer \"```"), - @ApiKeyAuthDefinition(name = DATABASE, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = DATABASE, - description = "Database context passed through http header. If no context is provided 'default' database " - + "context will be considered.")})) -@Path("/") -public class ReIngestionResource { - private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); - public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); - //TODO: Maximum number of concurrent re-ingestions allowed should be configurable - private static final int MAX_PARALLEL_REINGESTIONS = 10; - - // Map to track ongoing ingestion per segment - private static final ConcurrentHashMap SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); - - // Semaphore to enforce global concurrency limit - private static final Semaphore REINGESTION_SEMAPHORE = new Semaphore(MAX_PARALLEL_REINGESTIONS); - - @Inject - private ServerInstance _serverInstance; - - @POST - @Path("/reingestSegment") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " - + "upload the segment") - @ApiResponses(value = { - @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, - message = "Internal server error", response = ErrorInfo.class) - }) - public Response reIngestSegment(ReIngestionRequest request) { - try { - String tableNameWithType = request.getTableNameWithType(); - String segmentName = request.getSegmentName(); - - // Try to acquire a permit from the semaphore to ensure we don't exceed max concurrency - if (!REINGESTION_SEMAPHORE.tryAcquire()) { - return Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity("Too many re-ingestions in progress. Please try again later.") - .build(); - } - - // Check if the segment is already being re-ingested - AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); - if (!isIngesting.compareAndSet(false, true)) { - // The segment is already being ingested - REINGESTION_SEMAPHORE.release(); - return Response.status(Response.Status.CONFLICT) - .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") - .build(); - } - - InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); - if (instanceDataManager == null) { - throw new WebApplicationException(new RuntimeException("Invalid server initialization"), - Response.Status.INTERNAL_SERVER_ERROR); - } - - TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); - if (tableDataManager == null) { - throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } - - IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); - LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); - - // Get TableConfig and Schema - TableConfig tableConfig = indexLoadingConfig.getTableConfig(); - if (tableConfig == null) { - throw new WebApplicationException("Table config not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } - - Schema schema = indexLoadingConfig.getSchema(); - if (schema == null) { - throw new WebApplicationException("Schema not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } - - // Fetch SegmentZKMetadata - SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); - if (segmentZKMetadata == null) { - throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, - Response.Status.NOT_FOUND); - } - - // Get startOffset, endOffset, partitionGroupId - String startOffsetStr = segmentZKMetadata.getStartOffset(); - String endOffsetStr = segmentZKMetadata.getEndOffset(); - - if (startOffsetStr == null || endOffsetStr == null) { - return Response.serverError().entity("Start offset or end offset is null for segment: " + segmentName).build(); - } - - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); - int partitionGroupId = llcSegmentName.getPartitionGroupId(); - - Map streamConfigMap; - try { - streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); - } catch (Exception e) { - return Response.serverError().entity("Failed to get stream config for table: " + tableNameWithType).build(); - } - - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - - // Set up directories - File resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); - File resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); - - if (!resourceTmpDir.exists()) { - resourceTmpDir.mkdirs(); - } - if (!resourceDataDir.exists()) { - resourceDataDir.mkdirs(); - } - - LOGGER.info("Starting SimpleRealtimeSegmentDataManager..."); - // Instantiate SimpleRealtimeSegmentDataManager - SimpleRealtimeSegmentDataManager manager = - new SimpleRealtimeSegmentDataManager(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, - tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, resourceTmpDir, - resourceDataDir, _serverInstance.getServerMetrics()); - - try { - - manager.startConsumption(); - - waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300000, 0); - - manager.stopConsumption(); - - // After ingestion is complete, get the segment - if (!manager.isSuccess()) { - throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); - } - - LOGGER.info("Starting build for segment {}", segmentName); - SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = - manager.buildSegmentInternal(); - - // Get the segment directory - File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); - - if (segmentTarFile == null) { - throw new Exception("Failed to build segment: " + segmentName); - } - - ServerSegmentCompletionProtocolHandler protocolHandler = - new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); - - AuthProvider authProvider = protocolHandler.getAuthProvider(); - List
headers = AuthProviderUtils.toRequestHeaders(authProvider); - - String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); - - pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); - - LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); - // wait for segment metadata to have status as UPLOADED - waitForCondition((Void) -> { - SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); - if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { - return false; - } - - SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName); - return segmentDataManager instanceof ImmutableSegmentDataManager; - }, 5000, 300000, 0); - - // trigger segment reset call on API - LOGGER.info("Triggering segment reset for uploaded segment {}", segmentName); - HttpClient httpClient = HttpClient.getInstance(); - Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); - resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); - - LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); - } catch (Exception e) { - return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); - } finally { - // Clean up - manager.offload(); - manager.destroy(); - - // Delete temporary directories - FileUtils.deleteQuietly(resourceTmpDir); - FileUtils.deleteQuietly(resourceDataDir); - - isIngesting.set(false); - } - // Return success response - return Response.ok().entity(new ReIngestionResponse("Segment re-ingested and uploaded successfully")).build(); - } catch (Exception e) { - LOGGER.error("Error during re-ingestion", e); - throw new WebApplicationException(e, Response.Status.INTERNAL_SERVER_ERROR); - } finally { - REINGESTION_SEMAPHORE.release(); - } - } - - private void waitForCondition( - Function condition, long checkIntervalMs, long timeoutMs, long gracePeriodMs) { - long endTime = System.currentTimeMillis() + timeoutMs; - - // Adding grace period before starting the condition checks - if (gracePeriodMs > 0) { - LOGGER.info("Waiting for a grace period of {} ms before starting condition checks", gracePeriodMs); - try { - Thread.sleep(gracePeriodMs); - } catch (InterruptedException e) { - throw new RuntimeException("Interrupted during grace period wait", e); - } - } - - while (System.currentTimeMillis() < endTime) { - try { - if (Boolean.TRUE.equals(condition.apply(null))) { - LOGGER.info("Condition satisfied: {}", condition); - return; - } - Thread.sleep(checkIntervalMs); - } catch (Exception e) { - throw new RuntimeException("Caught exception while checking the condition", e); - } - } - - throw new RuntimeException("Timeout waiting for condition: " + condition); - } - - public void resetSegment(HttpClient httpClient, String controllerVipUrl, String tableNameWithType, String segmentName, - String targetInstance, Map headers) - throws IOException { - try { - //TODO: send correct headers - HttpClient.wrapAndThrowHttpException(httpClient.sendJsonPostRequest( - new URI(getURLForSegmentReset(controllerVipUrl, tableNameWithType, segmentName, targetInstance)), null, - headers)); - } catch (HttpErrorStatusException | URISyntaxException e) { - throw new IOException(e); - } - } - - private String getURLForSegmentReset(String controllerVipUrl, String tableNameWithType, String segmentName, - @Nullable String targetInstance) { - String query = targetInstance == null ? "reset" : "reset?targetInstance=" + targetInstance; - return StringUtil.join("/", controllerVipUrl, "segments", tableNameWithType, encode(segmentName), query); - } - - private String encode(String s) { - return URLEncoder.encode(s, StandardCharsets.UTF_8); - } - - /** - * Push segment metadata to the Pinot Controller in METADATA mode. - * - * @param tableNameWithType The table name with type (e.g., "myTable_OFFLINE") - * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") - * @param segmentFile The local segment tar.gz file - * @param authHeaders A map of authentication or additional headers for the request - */ - public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, - List
authHeaders, String segmentName, ServerSegmentCompletionProtocolHandler protocolHandler) - throws Exception { - LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), - tableNameWithType, controllerUrl); - String tableName = tableNameWithType; - File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); - - LOGGER.info("Generated segment metadata tar file: {}", segmentMetadataFile.getAbsolutePath()); - try { - // Prepare headers - List
headers = authHeaders; - - // The upload type must be METADATA - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, - FileUploadDownloadClient.FileUploadType.METADATA.toString())); - - // The DOWNLOAD_URI header specifies where the controller can fetch the segment if needed - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentFile.toURI().toString())); - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); - - // Set table name parameter - List parameters = getSegmentPushCommonParams(tableNameWithType); - - // Construct the endpoint URI - URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); - - LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); - - // Perform the metadata upload - SimpleHttpResponse response = protocolHandler.getFileUploadDownloadClient() - .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, - HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); - - LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, tableName, - controllerUrl, response.getStatusCode(), response.getResponse()); - } finally { - FileUtils.deleteQuietly(segmentMetadataFile); - } - } - - private List getSegmentPushCommonParams(String tableNameWithType) { - List params = new ArrayList<>(); - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION, - "true")); - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_NAME, - TableNameBuilder.extractRawTableName(tableNameWithType))); - TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); - if (tableType != null) { - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_TYPE, tableType.toString())); - } else { - throw new RuntimeException(String.format("Failed to determine the tableType from name: %s", tableNameWithType)); - } - return params; - } - - /** - * Generate a tar.gz file containing only the metadata files (metadata.properties, creation.meta) - * from a given Pinot segment tar.gz file. - */ - private File generateSegmentMetadataTar(File segmentTarFile) - throws Exception { - - if (!segmentTarFile.exists()) { - throw new IllegalArgumentException("Segment tar file does not exist: " + segmentTarFile.getAbsolutePath()); - } - - LOGGER.info("Generating segment metadata tar file from segment tar: {}", segmentTarFile.getAbsolutePath()); - File tempDir = Files.createTempDirectory("pinot-segment-temp").toFile(); - String uuid = UUID.randomUUID().toString(); - try { - File metadataDir = new File(tempDir, "segmentMetadataDir-" + uuid); - if (!metadataDir.mkdirs()) { - throw new RuntimeException("Failed to create metadata directory: " + metadataDir.getAbsolutePath()); - } - - LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", segmentTarFile, metadataDir); - TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, - new File(metadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); - - // Extract creation.meta - LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", segmentTarFile, metadataDir); - TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.SEGMENT_CREATION_META, - new File(metadataDir, V1Constants.SEGMENT_CREATION_META)); - - File segmentMetadataFile = - new File(FileUtils.getTempDirectory(), "segmentMetadata-" + UUID.randomUUID() + ".tar.gz"); - TarCompressionUtils.createCompressedTarFile(metadataDir, segmentMetadataFile); - return segmentMetadataFile; - } finally { - FileUtils.deleteQuietly(tempDir); - } - } - - private String getControllerUrl(String rawTableName, ServerSegmentCompletionProtocolHandler protocolHandler) { - ControllerLeaderLocator leaderLocator = ControllerLeaderLocator.getInstance(); - final Pair leaderHostPort = leaderLocator.getControllerLeader(rawTableName); - if (leaderHostPort == null) { - LOGGER.warn("No leader found for table: {}", rawTableName); - return null; - } - Integer port = leaderHostPort.getRight(); - String protocol = protocolHandler.getProtocol(); - Integer controllerHttpsPort = protocolHandler.getControllerHttpsPort(); - if (controllerHttpsPort != null) { - port = controllerHttpsPort; - protocol = HTTPS_PROTOCOL; - } - - return URIUtils.buildURI(protocol, leaderHostPort.getLeft() + ":" + port, "", Collections.emptyMap()).toString(); - } -} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java deleted file mode 100644 index a2b92379e549..000000000000 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * 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.server.api.resources.reingestion; - -public class ReIngestionRequest { - private String _tableNameWithType; - private String _segmentName; - - public String getTableNameWithType() { - return _tableNameWithType; - } - - public void setTableNameWithType(String tableNameWithType) { - _tableNameWithType = tableNameWithType; - } - - public String getSegmentName() { - return _segmentName; - } - - public void setSegmentName(String segmentName) { - _segmentName = segmentName; - } -} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java deleted file mode 100644 index af1b5f7d55bb..000000000000 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java +++ /dev/null @@ -1,36 +0,0 @@ -/** - * 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.server.api.resources.reingestion; - -public class ReIngestionResponse { - private String _message; - - public ReIngestionResponse(String message) { - _message = message; - } - - // Getter and setter - public String getMessage() { - return _message; - } - - public void setMessage(String message) { - _message = message; - } -} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java deleted file mode 100644 index 23b4b49e111c..000000000000 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ /dev/null @@ -1,569 +0,0 @@ -/** - * 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.server.api.resources.reingestion.utils; - -import com.google.common.annotations.VisibleForTesting; -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.commons.io.FileUtils; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metrics.ServerMetrics; -import org.apache.pinot.common.utils.FileUploadDownloadClient; -import org.apache.pinot.common.utils.TarCompressionUtils; -import org.apache.pinot.segment.local.data.manager.SegmentDataManager; -import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; -import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager; -import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; -import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; -import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; -import org.apache.pinot.segment.local.segment.creator.TransformPipeline; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.segment.local.utils.IngestionUtils; -import org.apache.pinot.segment.spi.MutableSegment; -import org.apache.pinot.segment.spi.V1Constants; -import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; -import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; -import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; -import org.apache.pinot.spi.config.table.ColumnPartitionConfig; -import org.apache.pinot.spi.config.table.SegmentPartitionConfig; -import org.apache.pinot.spi.config.table.SegmentZKPropsConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.plugin.PluginManager; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionGroupConsumer; -import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; -import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; -import org.apache.pinot.spi.stream.StreamDataDecoder; -import org.apache.pinot.spi.stream.StreamDataDecoderImpl; -import org.apache.pinot.spi.stream.StreamDataDecoderResult; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamMessageDecoder; -import org.apache.pinot.spi.stream.StreamMetadataProvider; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; -import org.apache.pinot.spi.utils.retry.RetryPolicies; -import org.apache.pinot.spi.utils.retry.RetryPolicy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * Simplified Segment Data Manager for ingesting data from a start offset to an end offset. - */ -public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { - - private static final int DEFAULT_CAPACITY = 100_000; - private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; - public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); - - private final String _segmentName; - private final String _tableNameWithType; - private final int _partitionGroupId; - private final String _segmentNameStr; - private final SegmentZKMetadata _segmentZKMetadata; - private final TableConfig _tableConfig; - private final Schema _schema; - private final StreamConfig _streamConfig; - private final StreamPartitionMsgOffsetFactory _offsetFactory; - private final StreamConsumerFactory _consumerFactory; - private StreamMetadataProvider _partitionMetadataProvider; - private final PartitionGroupConsumer _consumer; - private final StreamDataDecoder _decoder; - private final MutableSegmentImpl _realtimeSegment; - private final File _resourceTmpDir; - private final File _resourceDataDir; - private final Logger _logger; - private Thread _consumerThread; - private final AtomicBoolean _shouldStop = new AtomicBoolean(false); - private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false); - private final StreamPartitionMsgOffset _startOffset; - private final StreamPartitionMsgOffset _endOffset; - private volatile StreamPartitionMsgOffset _currentOffset; - private volatile int _numRowsIndexed = 0; - private final String _segmentStoreUriStr; - private final int _fetchTimeoutMs; - private final TransformPipeline _transformPipeline; - private volatile boolean _isSuccess = false; - private volatile Throwable _consumptionException; - private final ServerMetrics _serverMetrics; - - public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, - SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, - IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, - File resourceTmpDir, File resourceDataDir, ServerMetrics serverMetrics) - throws Exception { - - _segmentName = segmentName; - _tableNameWithType = tableNameWithType; - _partitionGroupId = partitionGroupId; - _segmentZKMetadata = segmentZKMetadata; - _tableConfig = tableConfig; - _schema = schema; - _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI(); - _streamConfig = streamConfig; - _resourceTmpDir = resourceTmpDir; - _resourceDataDir = resourceDataDir; - _serverMetrics = serverMetrics; - _logger = LoggerFactory.getLogger(SimpleRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); - - _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); - _startOffset = _offsetFactory.create(startOffsetStr); - _endOffset = _offsetFactory.create(endOffsetStr); - - String clientId = getClientId(); - - _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig); - _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId); - _segmentNameStr = _segmentZKMetadata.getSegmentName(); - - // Create a simple PartitionGroupConsumptionStatus - PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = - new PartitionGroupConsumptionStatus(_partitionGroupId, 0, _startOffset, null, null); - - _consumer = _consumerFactory.createPartitionGroupConsumer(clientId, partitionGroupConsumptionStatus); - - // Initialize decoder - Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig, _schema); - _decoder = createDecoder(fieldsToRead); - - // Fetch capacity from indexLoadingConfig or use default - int capacity = streamConfig.getFlushThresholdRows(); - if (capacity <= 0) { - capacity = DEFAULT_CAPACITY; - } - - // Fetch average number of multi-values from indexLoadingConfig - int avgNumMultiValues = indexLoadingConfig.getRealtimeAvgMultiValueCount(); - - // Load stats history, here we are using the same stats while as the RealtimeSegmentDataManager so that we are - // much more efficient in allocating buffers. It also works with empty file - String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != null - ? indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + File.separator + _tableNameWithType - : resourceTmpDir.getAbsolutePath(); - File statsHistoryFile = new File(tableDataDir, "segment-stats.ser"); - RealtimeSegmentStatsHistory statsHistory = RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile); - - // Initialize mutable segment with configurations - RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = - new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType) - .setSegmentName(_segmentName).setStreamName(_streamConfig.getTopicName()) - .setSegmentZKMetadata(_segmentZKMetadata).setStatsHistory(statsHistory).setSchema(_schema) - .setCapacity(capacity).setAvgNumMultiValues(avgNumMultiValues) - .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) - .setFieldConfigList(tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) - .setMemoryManager( - new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); - - setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); - - _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), _serverMetrics); - - _transformPipeline = new TransformPipeline(tableConfig, schema); - - // Initialize fetch timeout - _fetchTimeoutMs = - _streamConfig.getFetchTimeoutMillis() > 0 ? _streamConfig.getFetchTimeoutMillis() : DEFAULT_FETCH_TIMEOUT_MS; - } - - private String getClientId() { - return _tableNameWithType + "-" + _partitionGroupId; - } - - public void startConsumption() { - // Start the consumer thread - _consumerThread = new Thread(new PartitionConsumer(), _segmentName); - _consumerThread.start(); - } - - private StreamDataDecoder createDecoder(Set fieldsToRead) - throws Exception { - AtomicReference localStreamDataDecoder = new AtomicReference<>(); - RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetryPolicy(5, 1000L, 1.2f); - retryPolicy.attempt(() -> { - try { - StreamMessageDecoder streamMessageDecoder = createMessageDecoder(fieldsToRead); - localStreamDataDecoder.set(new StreamDataDecoderImpl(streamMessageDecoder)); - return true; - } catch (Exception e) { - _logger.warn("Failed to create StreamMessageDecoder. Retrying...", e); - return false; - } - }); - return localStreamDataDecoder.get(); - } - - /** - * Creates a {@link StreamMessageDecoder} using properties in {@link StreamConfig}. - * - * @param fieldsToRead The fields to read from the source stream - * @return The initialized StreamMessageDecoder - */ - private StreamMessageDecoder createMessageDecoder(Set fieldsToRead) { - String decoderClass = _streamConfig.getDecoderClass(); - try { - StreamMessageDecoder decoder = PluginManager.get().createInstance(decoderClass); - decoder.init(fieldsToRead, _streamConfig, _tableConfig, _schema); - return decoder; - } catch (Exception e) { - throw new RuntimeException( - "Caught exception while creating StreamMessageDecoder from stream config: " + _streamConfig, e); - } - } - - private class PartitionConsumer implements Runnable { - @Override - public void run() { - try { - _consumer.start(_startOffset); - _currentOffset = _startOffset; - TransformPipeline.Result reusedResult = new TransformPipeline.Result(); - while (!_shouldStop.get() && _currentOffset.compareTo(_endOffset) < 0) { - // Fetch messages - MessageBatch messageBatch = _consumer.fetchMessages(_currentOffset, _fetchTimeoutMs); - - int messageCount = messageBatch.getMessageCount(); - - for (int i = 0; i < messageCount; i++) { - if (_shouldStop.get()) { - break; - } - StreamMessage streamMessage = messageBatch.getStreamMessage(i); - if (streamMessage.getMetadata() != null && streamMessage.getMetadata().getOffset() != null - && streamMessage.getMetadata().getOffset().compareTo(_endOffset) >= 0) { - _shouldStop.set(true); - _logger.info("Reached end offset: {} for partition group: {}", _endOffset, _partitionGroupId); - break; - } - - // Decode message - StreamDataDecoderResult decodedResult = _decoder.decode(streamMessage); - if (decodedResult.getException() == null) { - // Index message - GenericRow row = decodedResult.getResult(); - - _transformPipeline.processRow(row, reusedResult); - - List transformedRows = reusedResult.getTransformedRows(); - - // TODO: Do enrichment and transforms before indexing - for (GenericRow transformedRow : transformedRows) { - _realtimeSegment.index(transformedRow, streamMessage.getMetadata()); - _numRowsIndexed++; - } - } else { - _logger.warn("Failed to decode message at offset {}: {}", _currentOffset, decodedResult.getException()); - } - } - - _currentOffset = messageBatch.getOffsetOfNextBatch(); - } - _isSuccess = true; - } catch (Exception e) { - _logger.error("Exception in consumer thread", e); - _consumptionException = e; - throw new RuntimeException(e); - } finally { - try { - _consumer.close(); - } catch (Exception e) { - _logger.warn("Failed to close consumer", e); - } - _isDoneConsuming.set(true); - } - } - } - - public void stopConsumption() { - _shouldStop.set(true); - if (_consumerThread.isAlive()) { - _consumerThread.interrupt(); - try { - _consumerThread.join(); - } catch (InterruptedException e) { - _logger.warn("Interrupted while waiting for consumer thread to finish"); - } - } - } - - @Override - public MutableSegment getSegment() { - return _realtimeSegment; - } - - @Override - public String getSegmentName() { - return _segmentName; - } - - @Override - protected void doDestroy() { - _realtimeSegment.destroy(); - } - - @Override - public void doOffload() { - stopConsumption(); - } - - public boolean isDoneConsuming() { - return _isDoneConsuming.get(); - } - - public boolean isSuccess() { - return _isSuccess; - } - - public Throwable getConsumptionException() { - return _consumptionException; - } - - @VisibleForTesting - public SegmentBuildDescriptor buildSegmentInternal() throws Exception { - _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); - final long lockAcquireTimeMillis = now(); - // Build a segment from in-memory rows. - // Use a temporary directory - Path tempSegmentFolder = null; - try { - tempSegmentFolder = - java.nio.file.Files.createTempDirectory(_resourceTmpDir.toPath(), "tmp-" + _segmentNameStr + "-"); - } catch (IOException e) { - _logger.error("Failed to create temporary directory for segment build", e); - return null; - } - - SegmentZKPropsConfig segmentZKPropsConfig = new SegmentZKPropsConfig(); - segmentZKPropsConfig.setStartOffset(_startOffset.toString()); - segmentZKPropsConfig.setEndOffset(_endOffset.toString()); - - // Build the segment - RealtimeSegmentConverter converter = - new RealtimeSegmentConverter(_realtimeSegment, segmentZKPropsConfig, tempSegmentFolder.toString(), - _schema, _tableNameWithType, _tableConfig, _segmentZKMetadata.getSegmentName(), - _tableConfig.getIndexingConfig().isNullHandlingEnabled()); - try { - converter.build(null, _serverMetrics); - } catch (Exception e) { - _logger.error("Failed to build segment", e); - FileUtils.deleteQuietly(tempSegmentFolder.toFile()); - return null; - } - final long buildTimeMillis = now() - lockAcquireTimeMillis; - - File dataDir = _resourceDataDir; - File indexDir = new File(dataDir, _segmentNameStr); - FileUtils.deleteQuietly(indexDir); - - File tempIndexDir = new File(tempSegmentFolder.toFile(), _segmentNameStr); - if (!tempIndexDir.exists()) { - _logger.error("Temp index directory {} does not exist", tempIndexDir); - FileUtils.deleteQuietly(tempSegmentFolder.toFile()); - return null; - } - try { - FileUtils.moveDirectory(tempIndexDir, indexDir); - } catch (IOException e) { - _logger.error("Caught exception while moving index directory from: {} to: {}", tempIndexDir, indexDir, e); - return null; - } finally { - FileUtils.deleteQuietly(tempSegmentFolder.toFile()); - } - - SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); - - long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); - File segmentTarFile = new File(dataDir, _segmentNameStr + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); - try { - TarCompressionUtils.createCompressedTarFile(indexDir, segmentTarFile); - } catch (IOException e) { - _logger.error("Caught exception while tarring index directory from: {} to: {}", indexDir, segmentTarFile, e); - return null; - } - - File metadataFile = SegmentDirectoryPaths.findMetadataFile(indexDir); - if (metadataFile == null) { - _logger.error("Failed to find metadata file under index directory: {}", indexDir); - return null; - } - File creationMetaFile = SegmentDirectoryPaths.findCreationMetaFile(indexDir); - if (creationMetaFile == null) { - _logger.error("Failed to find creation meta file under index directory: {}", indexDir); - return null; - } - Map metadataFiles = new HashMap<>(); - metadataFiles.put(V1Constants.MetadataKeys.METADATA_FILE_NAME, metadataFile); - metadataFiles.put(V1Constants.SEGMENT_CREATION_META, creationMetaFile); - return new SegmentBuildDescriptor(segmentTarFile, metadataFiles, _currentOffset, buildTimeMillis, buildTimeMillis, - segmentSizeBytes, segmentMetadata); - } - - protected long now() { - return System.currentTimeMillis(); - } - - public void removeSegmentFile(SegmentBuildDescriptor segmentBuildDescriptor) { - if (segmentBuildDescriptor != null) { - segmentBuildDescriptor.deleteSegmentFile(); - } - } - - /* - * set the following partition parameters in RT segment config builder: - * - partition column - * - partition function - * - partition group id - */ - private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder, - SegmentPartitionConfig segmentPartitionConfig) { - if (segmentPartitionConfig != null) { - Map columnPartitionMap = segmentPartitionConfig.getColumnPartitionMap(); - if (columnPartitionMap.size() == 1) { - Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); - String partitionColumn = entry.getKey(); - ColumnPartitionConfig columnPartitionConfig = entry.getValue(); - String partitionFunctionName = columnPartitionConfig.getFunctionName(); - - // NOTE: Here we compare the number of partitions from the config and the stream, and log a warning and emit a - // metric when they don't match, but use the one from the stream. The mismatch could happen when the - // stream partitions are changed, but the table config has not been updated to reflect the change. - // In such case, picking the number of partitions from the stream can keep the segment properly - // partitioned as long as the partition function is not changed. - int numPartitions = columnPartitionConfig.getNumPartitions(); - try { - // TODO: currentPartitionGroupConsumptionStatus should be fetched from idealState + segmentZkMetadata, - // so that we get back accurate partitionGroups info - // However this is not an issue for Kafka, since partitionGroups never expire and every partitionGroup has - // a single partition - // Fix this before opening support for partitioning in Kinesis - int numPartitionGroups = - _partitionMetadataProvider.computePartitionGroupMetadata(getClientId(), _streamConfig, - Collections.emptyList(), /*maxWaitTimeMs=*/5000).size(); - - if (numPartitionGroups != numPartitions) { - _logger.info( - "Number of stream partitions: {} does not match number of partitions in the partition config: {}, " - + "using number of stream " + "partitions", numPartitionGroups, numPartitions); - numPartitions = numPartitionGroups; - } - } catch (Exception e) { - _logger.warn("Failed to get number of stream partitions in 5s, " - + "using number of partitions in the partition config: {}", numPartitions, e); - createPartitionMetadataProvider("Timeout getting number of stream partitions"); - } - - realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); - realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, null)); - realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); - } else { - _logger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); - } - } - } - - /** - * Creates a new stream metadata provider - */ - private void createPartitionMetadataProvider(String reason) { - closePartitionMetadataProvider(); - _logger.info("Creating new partition metadata provider, reason: {}", reason); - _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(getClientId(), _partitionGroupId); - } - - private void closePartitionMetadataProvider() { - if (_partitionMetadataProvider != null) { - try { - _partitionMetadataProvider.close(); - } catch (Exception e) { - _logger.warn("Could not close stream metadata provider", e); - } - } - } - - public class SegmentBuildDescriptor { - final File _segmentTarFile; - final Map _metadataFileMap; - final StreamPartitionMsgOffset _offset; - final long _waitTimeMillis; - final long _buildTimeMillis; - final long _segmentSizeBytes; - final SegmentMetadataImpl _segmentMetadata; - - public SegmentBuildDescriptor(@Nullable File segmentTarFile, @Nullable Map metadataFileMap, - StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, - SegmentMetadataImpl segmentMetadata) { - _segmentTarFile = segmentTarFile; - _metadataFileMap = metadataFileMap; - _offset = _offsetFactory.create(offset); - _buildTimeMillis = buildTimeMillis; - _waitTimeMillis = waitTimeMillis; - _segmentSizeBytes = segmentSizeBytes; - _segmentMetadata = segmentMetadata; - } - - public StreamPartitionMsgOffset getOffset() { - return _offset; - } - - public long getBuildTimeMillis() { - return _buildTimeMillis; - } - - public long getWaitTimeMillis() { - return _waitTimeMillis; - } - - @Nullable - public File getSegmentTarFile() { - return _segmentTarFile; - } - - @Nullable - public Map getMetadataFiles() { - return _metadataFileMap; - } - - public long getSegmentSizeBytes() { - return _segmentSizeBytes; - } - - public void deleteSegmentFile() { - if (_segmentTarFile != null) { - FileUtils.deleteQuietly(_segmentTarFile); - } - } - - public SegmentMetadataImpl getSegmentMetadata() { - return _segmentMetadata; - } - } -} From aee514c0bfd0b7298dac0f70b9aae337c3346b50 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Thu, 23 Jan 2025 14:01:23 +0530 Subject: [PATCH 30/65] Removing check in replaceSegmentIfCrcMismatch --- .../data/manager/BaseTableDataManager.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 4111756a76d2..c1462ec5b9a5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -56,7 +56,6 @@ import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; -import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; @@ -385,20 +384,15 @@ protected void replaceSegmentIfCrcMismatch(SegmentDataManager segmentDataManager IndexLoadingConfig indexLoadingConfig) throws Exception { String segmentName = segmentDataManager.getSegmentName(); - TableConfig tableConfig = indexLoadingConfig.getTableConfig(); - // For pauseless tables, we should replace the segment if download url is missing even if crc is same - if (!PauselessConsumptionUtils.isPauselessEnabled(tableConfig)) { - Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, - "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); - SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); - if (hasSameCRC(zkMetadata, localMetadata)) { - _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); - return; - } - _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, - localMetadata.getCrc(), zkMetadata.getCrc()); + Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, + "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); + SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); + if (hasSameCRC(zkMetadata, localMetadata)) { + _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); + return; } - + _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, + localMetadata.getCrc(), zkMetadata.getCrc()); downloadAndLoadSegment(zkMetadata, indexLoadingConfig); _logger.info("Replaced segment: {} with new CRC: {}", segmentName, zkMetadata.getCrc()); } From 88a619a4b698a00be5a81abf858d0df20584a7ba Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Fri, 24 Jan 2025 12:48:50 +0530 Subject: [PATCH 31/65] Adding a new class for simple serialization and deserialization of major fields of SegmentZKMetadata --- .../segment/SegmentZKMetadataUtils.java | 81 -------- .../segment/SimpleSegmentMetadata.java | 174 ++++++++++++++++++ .../utils/FileUploadDownloadClient.java | 7 +- .../segment/SegmentZKMetadataUtilsTest.java | 126 ------------- .../PinotLLCRealtimeSegmentManager.java | 8 +- .../PinotLLCRealtimeSegmentManagerTest.java | 3 +- .../server/api/resources/TablesResource.java | 4 +- 7 files changed, 187 insertions(+), 216 deletions(-) delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java delete mode 100644 pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java deleted file mode 100644 index d69bb38adc81..000000000000 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * 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.common.metadata.segment; - -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.MapperFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; -import com.fasterxml.jackson.databind.node.ObjectNode; -import java.io.IOException; -import org.apache.helix.zookeeper.datamodel.ZNRecord; - - -public class SegmentZKMetadataUtils { - private SegmentZKMetadataUtils() { - } - - public static final ObjectMapper MAPPER = createObjectMapper(); - - private static ObjectMapper createObjectMapper() { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(SerializationFeature.INDENT_OUTPUT, true); - mapper.configure(MapperFeature.AUTO_DETECT_FIELDS, true); - mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, true); - mapper.configure(MapperFeature.CAN_OVERRIDE_ACCESS_MODIFIERS, true); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return mapper; - } - - public static String serialize(SegmentZKMetadata metadata) - throws IOException { - if (metadata == null) { - return null; - } - return MAPPER.writeValueAsString(metadata.toZNRecord()); - } - - public static SegmentZKMetadata deserialize(String jsonString) - throws IOException { - if (jsonString == null || jsonString.isEmpty()) { - return null; - } - ObjectNode objectNode = (ObjectNode) MAPPER.readTree(jsonString); - ZNRecord znRecord = MAPPER.treeToValue(objectNode, ZNRecord.class); - return new SegmentZKMetadata(znRecord); - } - - public static SegmentZKMetadata deserialize(ObjectNode objectNode) - throws IOException { - if (objectNode == null) { - return null; - } - ZNRecord znRecord = MAPPER.treeToValue(objectNode, ZNRecord.class); - return new SegmentZKMetadata(znRecord); - } - - public static SegmentZKMetadata deserialize(byte[] bytes) - throws IOException { - if (bytes == null || bytes.length == 0) { - return null; - } - ZNRecord znRecord = MAPPER.readValue(bytes, ZNRecord.class); - return new SegmentZKMetadata(znRecord); - } -} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java new file mode 100644 index 000000000000..5d009bec96a4 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java @@ -0,0 +1,174 @@ +/** + * 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.common.metadata.segment; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonGetter; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import org.apache.pinot.spi.utils.CommonConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A simplified version of SegmentZKMetadata designed for easy serialization and deserialization. + * This class maintains only the essential fields from SegmentZKMetadata while providing JSON + * serialization support via Jackson annotations. + * + * Use cases: + * 1. Serializing segment metadata for API responses + * 2. Transferring segment metadata between services + * 3. Storing segment metadata in a format that's easy to deserialize + * + * Example usage: + * SimpleSegmentMetadata metadata = SimpleSegmentMetadata.fromZKMetadata(zkMetadata); + * String json = JsonUtils.objectToString(metadata); + * SimpleSegmentMetadata deserialized = objectMapper.readValue(json, SimpleSegmentMetadata.class); + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class SimpleSegmentMetadata { + private static final Logger LOGGER = LoggerFactory.getLogger(SimpleSegmentMetadata.class); + private final String _segmentName; + private final Map _simpleFields; + + @JsonCreator + public SimpleSegmentMetadata( + @JsonProperty("segmentName") String segmentName, + @JsonProperty("simpleFields") Map simpleFields) { + _segmentName = segmentName; + _simpleFields = new HashMap<>(simpleFields); + } + + @JsonGetter + public String getSegmentName() { + return _segmentName; + } + + @JsonGetter + public Map getSimpleFields() { + return Collections.unmodifiableMap(_simpleFields); + } + + public long getStartTimeMs() { + long startTimeMs = -1; + String startTimeString = _simpleFields.get(CommonConstants.Segment.START_TIME); + if (startTimeString != null) { + long startTime = Long.parseLong(startTimeString); + if (startTime > 0) { + // NOTE: Need to check whether the start time is positive because some old segment ZK metadata contains negative + // start time and null time unit + startTimeMs = TimeUnit.valueOf(_simpleFields.get(CommonConstants.Segment.TIME_UNIT)).toMillis(startTime); + } + } + return startTimeMs; + } + + public long getEndTimeMs() { + long endTimeMs = -1; + String endTimeString = _simpleFields.get(CommonConstants.Segment.END_TIME); + if (endTimeString != null) { + long endTime = Long.parseLong(endTimeString); + // NOTE: Need to check whether the end time is positive because some old segment ZK metadata contains negative + // end time and null time unit + if (endTime > 0) { + endTimeMs = TimeUnit.valueOf(_simpleFields.get(CommonConstants.Segment.TIME_UNIT)).toMillis(endTime); + } + } + return endTimeMs; + } + + public String getIndexVersion() { + return _simpleFields.get(CommonConstants.Segment.INDEX_VERSION); + } + + public long getTotalDocs() { + String value = _simpleFields.get(CommonConstants.Segment.TOTAL_DOCS); + return value != null ? Long.parseLong(value) : -1; + } + + public SegmentPartitionMetadata getPartitionMetadata() { + String partitionMetadataJson = _simpleFields.get(CommonConstants.Segment.PARTITION_METADATA); + if (partitionMetadataJson != null) { + try { + return SegmentPartitionMetadata.fromJsonString(partitionMetadataJson); + } catch (Exception e) { + LOGGER.error("Caught exception while reading partition metadata for segment: {}", getSegmentName(), e); + } + } + return null; + } + + public long getSizeInBytes() { + String value = _simpleFields.get(CommonConstants.Segment.SIZE_IN_BYTES); + return value != null ? Long.parseLong(value) : -1; + } + + public long getCrc() { + String value = _simpleFields.get(CommonConstants.Segment.CRC); + return value != null ? Long.parseLong(value) : -1; + } + + public String getDownloadUrl() { + String downloadUrl = _simpleFields.get(CommonConstants.Segment.DOWNLOAD_URL); + // Handle legacy download url keys + if (downloadUrl == null) { + downloadUrl = _simpleFields.get(CommonConstants.Segment.Offline.DOWNLOAD_URL); + if (downloadUrl == null) { + downloadUrl = _simpleFields.get(CommonConstants.Segment.Realtime.DOWNLOAD_URL); + } + } + return downloadUrl; + } + + /** + * Creates a SimpleSegmentMetadata instance from a SegmentZKMetadata object. + * This method copies all simple fields from the ZK metadata while maintaining + * the immutability guarantees of this class. + */ + public static SimpleSegmentMetadata fromZKMetadata(SegmentZKMetadata zkMetadata) { + return new SimpleSegmentMetadata( + zkMetadata.getSegmentName(), + zkMetadata.toMap() + ); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SimpleSegmentMetadata that = (SimpleSegmentMetadata) o; + return Objects.equals(_segmentName, that._segmentName) + && Objects.equals(_simpleFields, that._simpleFields); + } + + @Override + public int hashCode() { + return Objects.hash(_segmentName, _simpleFields); + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 8bf5e576c005..7260a98155d7 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -56,7 +56,7 @@ import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; +import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.restlet.resources.EndReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.StartReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; @@ -1001,13 +1001,14 @@ public TableLLCSegmentUploadResponse uploadLLCToSegmentStore(String uri) * @throws IOException * @throws HttpErrorStatusException */ - public SegmentZKMetadata uploadLLCToSegmentStoreWithZKMetadata(String uri) + public SimpleSegmentMetadata uploadLLCToSegmentStoreWithZKMetadata(String uri) throws URISyntaxException, IOException, HttpErrorStatusException { ClassicRequestBuilder requestBuilder = ClassicRequestBuilder.post(new URI(uri)).setVersion(HttpVersion.HTTP_1_1); // sendRequest checks the response status code SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); - SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize(response.getResponse()); + SimpleSegmentMetadata segmentZKMetadata = + JsonUtils.stringToObject(response.getResponse(), SimpleSegmentMetadata.class); if (segmentZKMetadata.getDownloadUrl() == null || segmentZKMetadata.getDownloadUrl().isEmpty()) { throw new HttpErrorStatusException( diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java deleted file mode 100644 index 6122b997aeb5..000000000000 --- a/pinot-common/src/test/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtilsTest.java +++ /dev/null @@ -1,126 +0,0 @@ -/** - * 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.common.metadata.segment; - -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.ObjectNode; -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static org.testng.Assert.*; - -public class SegmentZKMetadataUtilsTest { - - private SegmentZKMetadata _testMetadata; - private static final String TEST_SEGMENT_NAME = "testSegment"; - private static final String TEST_TABLE_SEGMENT = "mytable__0__0__20220722T2342Z"; - - @BeforeMethod - public void setUp() { - // Create a test metadata object with sample data - _testMetadata = new SegmentZKMetadata(TEST_SEGMENT_NAME); - _testMetadata.setStartTime(1234567890L); - _testMetadata.setEndTime(1234567899L); - _testMetadata.setTimeUnit(TimeUnit.SECONDS); - _testMetadata.setIndexVersion("v1"); - _testMetadata.setTotalDocs(1000); - _testMetadata.setSizeInBytes(1024 * 1024); - _testMetadata.setCrc(123456L); - _testMetadata.setCreationTime(System.currentTimeMillis()); - } - - @Test - public void testSerialize() throws IOException { - // Test successful serialization - String serialized = SegmentZKMetadataUtils.serialize(_testMetadata); - - // Verify basic properties - assertNotNull(serialized, "Serialized string should not be null"); - assertTrue(serialized.contains(TEST_SEGMENT_NAME), "Serialized string should contain segment name"); - assertTrue(serialized.contains("SECONDS"), "Serialized string should contain time unit"); - - // Verify JSON structure - ObjectNode jsonNode = (ObjectNode) SegmentZKMetadataUtils.MAPPER.readTree(serialized); - assertTrue(jsonNode.has("simpleFields"), "Should contain simpleFields"); - assertTrue(jsonNode.has("mapFields"), "Should contain mapFields"); - } - - @Test - public void testSerializeNull() throws IOException { - assertNull(SegmentZKMetadataUtils.serialize(null), "Serializing null should return null"); - } - - @Test - public void testDeserializeString() throws IOException { - String errorStr = "{\"id\":\"" + TEST_TABLE_SEGMENT + "\",\"simpleFields\":" - + "{\"segment.crc\":\"2624963047\",\"segment.creation.time\":\"1658533353347\"," - + "\"segment.download.url\":\"http://localhost:18998/segments/mytable/" + TEST_TABLE_SEGMENT + "\"," - + "\"segment.end.time\":\"1405296000000\",\"segment.flush.threshold.size\":\"2500\"," - + "\"segment.index.version\":\"v3\",\"segment.realtime.endOffset\":\"2500\"," - + "\"segment.realtime.numReplicas\":\"1\",\"segment.realtime.startOffset\":\"0\"," - + "\"segment.realtime.status\":\"DONE\",\"segment.start.time\":\"1404086400000\"," - + "\"segment.time.unit\":\"MILLISECONDS\",\"segment.total.docs\":\"2500\"}," - + "\"mapFields\":{},\"listFields\":{}}"; - - SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize(errorStr); - - // Verify deserialized properties - assertEquals(segmentZKMetadata.getSegmentName(), TEST_TABLE_SEGMENT, - "Segment name should match expected value"); - assertEquals(segmentZKMetadata.getEndTimeMs(), 1405296000000L, - "End time should match expected value"); - assertEquals(segmentZKMetadata.getCrc(), 2624963047L, - "CRC should match expected value"); - } - - @Test - public void testDeserializeObjectNode() throws IOException { - String errorStr = "{\"id\":\"" + TEST_TABLE_SEGMENT + "\",\"simpleFields\":" - + "{\"segment.crc\":\"2624963047\",\"segment.creation.time\":\"1658533353347\"," - + "\"segment.download.url\":\"http://localhost:18998/segments/mytable/" + TEST_TABLE_SEGMENT + "\"," - + "\"segment.end.time\":\"1405296000000\",\"segment.flush.threshold.size\":\"2500\"," - + "\"segment.index.version\":\"v3\",\"segment.realtime.endOffset\":\"2500\"," - + "\"segment.realtime.numReplicas\":\"1\",\"segment.realtime.startOffset\":\"0\"," - + "\"segment.realtime.status\":\"DONE\",\"segment.start.time\":\"1404086400000\"," - + "\"segment.time.unit\":\"MILLISECONDS\",\"segment.total.docs\":\"2500\"}," - + "\"mapFields\":{},\"listFields\":{}}"; - - JsonNode zkChildren = SegmentZKMetadataUtils.MAPPER.readTree(errorStr); - SegmentZKMetadata segmentZKMetadata = SegmentZKMetadataUtils.deserialize((ObjectNode) zkChildren); - - assertEquals(segmentZKMetadata.getSegmentName(), TEST_TABLE_SEGMENT, - "Segment name should match expected value"); - assertEquals(segmentZKMetadata.getEndTimeMs(), 1405296000000L, - "End time should match expected value"); - } - - @Test - public void testDeserializeBytes() throws IOException { - String serialized = SegmentZKMetadataUtils.serialize(_testMetadata); - byte[] bytes = serialized.getBytes(); - - SegmentZKMetadata deserialized = SegmentZKMetadataUtils.deserialize(bytes); - - assertNotNull(deserialized, "Deserialized object should not be null"); - assertEquals(deserialized.getSegmentName(), _testMetadata.getSegmentName(), - "Segment names should match"); - } -} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 4710a2c8aa8b..00ee7c70bad3 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -61,6 +61,7 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.metrics.ControllerGauge; import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; @@ -1778,7 +1779,7 @@ private void uploadToDeepStoreWithFallback(URI uri, String segmentName, String r String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegmentToDeepStore"); LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, serverUploadRequestUrl); - SegmentZKMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( + SimpleSegmentMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( serverUploadRequestUrl); handleMetadataUpload(segmentName, rawTableName, segmentZKMetadata, uploadedMetadata, pinotFS); }, @@ -1824,7 +1825,7 @@ private String getUploadUrl(URI uri, String endpoint) { } private void handleMetadataUpload(String segmentName, String rawTableName, - SegmentZKMetadata currentMetadata, SegmentZKMetadata uploadedMetadata, + SegmentZKMetadata currentMetadata, SimpleSegmentMetadata uploadedMetadata, PinotFS pinotFS) throws Exception { @@ -1877,7 +1878,8 @@ private void handleBasicUpload(String segmentName, String rawTableName, * @param segmentZKMetadata Current segment metadata stored in ZooKeeper that needs to be updated * @param uploadedSegmentZKMetadata New metadata from the successfully uploaded segment */ - private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata uploadedSegmentZKMetadata) { + private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, + SimpleSegmentMetadata uploadedSegmentZKMetadata) { if (segmentZKMetadata.getStatus() == Status.COMMITTING) { LOGGER.info("Updating additional metadata in ZK for segment {} as pauseless is enabled", segmentZKMetadata.getSegmentName()); diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 84d70232ca7c..19f2d616f4cf 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -50,6 +50,7 @@ import org.apache.pinot.common.assignment.InstancePartitions; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; @@ -1248,7 +1249,7 @@ public void testUploadLLCSegmentToDeepStore() segmentZKMetadataCopy.setDownloadUrl(tempSegmentFileLocation.getPath()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( serverUploadRequestUrl0)).thenReturn( - segmentZKMetadataCopy); + SimpleSegmentMetadata.fromZKMetadata(segmentZKMetadataCopy)); // Change 2nd segment status to be DONE, but with default peer download url. // Verify later the download url isn't fixed after upload failure. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 60e568acf5a4..92c522e8a50c 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -65,7 +65,7 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; +import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; @@ -1065,7 +1065,7 @@ public String uploadLLCSegmentToDeepStore( segmentName), Response.Status.INTERNAL_SERVER_ERROR); } segmentZKMetadata.setDownloadUrl(segmentDownloadUrl.toString()); - return SegmentZKMetadataUtils.serialize(segmentZKMetadata); + return JsonUtils.objectToPrettyString(SimpleSegmentMetadata.fromZKMetadata(segmentZKMetadata)); } finally { FileUtils.deleteQuietly(segmentTarFile); tableDataManager.releaseSegment(segmentDataManager); From 791ac210b0e1c503160cb0542af6b42e0fb8a7cc Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Fri, 24 Jan 2025 13:04:14 +0530 Subject: [PATCH 32/65] Removing files related to reingestion tests --- ...reInjectingRealtimeSegmentDataManager.java | 66 ------------- ...lureInjectingRealtimeTableDataManager.java | 69 -------------- ...lureInjectingTableDataManagerProvider.java | 93 ------------------- 3 files changed, 228 deletions(-) delete mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java delete mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java delete mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java deleted file mode 100644 index 072cc92e4923..000000000000 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * 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.integration.tests.realtime; - -import java.util.concurrent.Semaphore; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metrics.ServerMetrics; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; -import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.utils.retry.AttemptsExceededException; -import org.apache.pinot.spi.utils.retry.RetriableOperationException; - - -/** - * A specialized RealtimeSegmentDataManager that lets us inject a forced failure - * in the commit step, which occurs strictly after the segmentConsumed message. - */ -public class FailureInjectingRealtimeSegmentDataManager extends RealtimeSegmentDataManager { - - // This flag controls whether commit should forcibly fail. - private final boolean _failCommit; - - /** - * Creates a manager that will forcibly fail the commit segment step. - */ - public FailureInjectingRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, - TableConfig tableConfig, RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, - IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, - Semaphore partitionGroupConsumerSemaphore, ServerMetrics serverMetrics, - boolean failCommit) throws AttemptsExceededException, RetriableOperationException { - // Pass through to the real parent constructor - super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, - indexLoadingConfig, schema, llcSegmentName, partitionGroupConsumerSemaphore, serverMetrics, - null /* no PartitionUpsertMetadataManager */, null /* no PartitionDedupMetadataManager */, - () -> true /* isReadyToConsumeData always true for tests */); - - _failCommit = failCommit; - } - - protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { - if (_failCommit) { - throw new RuntimeException("Forced failure in buildSegmentInternal"); - } - return super.buildSegmentInternal(forCommit); - } -} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java deleted file mode 100644 index 7dfd703bd98a..000000000000 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * 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.integration.tests.realtime; - -import java.util.concurrent.Semaphore; -import java.util.function.BooleanSupplier; -import java.util.function.Supplier; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.PauselessConsumptionUtils; -import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; -import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; -import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.segment.local.upsert.PartitionUpsertMetadataManager; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.utils.retry.AttemptsExceededException; -import org.apache.pinot.spi.utils.retry.RetriableOperationException; - - -public class FailureInjectingRealtimeTableDataManager extends RealtimeTableDataManager { - private volatile boolean _hasFailedOnce = false; - - public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore) { - this(segmentBuildSemaphore, () -> true); - } - - public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore, - Supplier isServerReadyToServeQueries) { - super(segmentBuildSemaphore, isServerReadyToServeQueries); - } - - @Override - protected RealtimeSegmentDataManager createRealtimeSegmentDataManager(SegmentZKMetadata zkMetadata, - TableConfig tableConfig, IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, - Semaphore semaphore, PartitionUpsertMetadataManager partitionUpsertMetadataManager, - PartitionDedupMetadataManager partitionDedupMetadataManager, BooleanSupplier isTableReadyToConsumeData) - throws AttemptsExceededException, RetriableOperationException { - - boolean addFailureToCommits = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); - - if (addFailureToCommits) { - if (_hasFailedOnce) { - addFailureToCommits = false; - } else { - _hasFailedOnce = true; - } - } - return new FailureInjectingRealtimeSegmentDataManager(zkMetadata, tableConfig, this, _indexDir.getAbsolutePath(), - indexLoadingConfig, schema, llcSegmentName, semaphore, _serverMetrics, addFailureToCommits); - } -} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java deleted file mode 100644 index 0c323320d51a..000000000000 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java +++ /dev/null @@ -1,93 +0,0 @@ -/** - * 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.integration.tests.realtime; - -import com.google.common.cache.Cache; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Semaphore; -import java.util.function.Supplier; -import javax.annotation.Nullable; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.helix.HelixManager; -import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; -import org.apache.pinot.core.data.manager.offline.DimensionTableDataManager; -import org.apache.pinot.core.data.manager.offline.OfflineTableDataManager; -import org.apache.pinot.core.data.manager.provider.TableDataManagerProvider; -import org.apache.pinot.segment.local.data.manager.TableDataManager; -import org.apache.pinot.segment.local.utils.SegmentLocks; -import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.utils.CommonConstants; -import org.apache.pinot.spi.utils.IngestionConfigUtils; - - -/** - * Default implementation of {@link TableDataManagerProvider}. - */ -public class FailureInjectingTableDataManagerProvider implements TableDataManagerProvider { - private InstanceDataManagerConfig _instanceDataManagerConfig; - private HelixManager _helixManager; - private SegmentLocks _segmentLocks; - private Semaphore _segmentBuildSemaphore; - - @Override - public void init(InstanceDataManagerConfig instanceDataManagerConfig, HelixManager helixManager, - SegmentLocks segmentLocks) { - _instanceDataManagerConfig = instanceDataManagerConfig; - _helixManager = helixManager; - _segmentLocks = segmentLocks; - int maxParallelSegmentBuilds = instanceDataManagerConfig.getMaxParallelSegmentBuilds(); - _segmentBuildSemaphore = maxParallelSegmentBuilds > 0 ? new Semaphore(maxParallelSegmentBuilds, true) : null; - } - - @Override - public TableDataManager getTableDataManager(TableConfig tableConfig, @Nullable ExecutorService segmentPreloadExecutor, - @Nullable Cache, SegmentErrorInfo> errorCache, - Supplier isServerReadyToServeQueries) { - TableDataManager tableDataManager; - switch (tableConfig.getTableType()) { - case OFFLINE: - if (tableConfig.isDimTable()) { - tableDataManager = DimensionTableDataManager.createInstanceByTableName(tableConfig.getTableName()); - } else { - tableDataManager = new OfflineTableDataManager(); - } - break; - case REALTIME: - Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); - if (Boolean.parseBoolean(streamConfigMap.get(StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE)) - && StringUtils.isEmpty(_instanceDataManagerConfig.getSegmentStoreUri())) { - throw new IllegalStateException(String.format("Table has enabled %s config. But the server has not " - + "configured the segmentstore uri. Configure the server config %s", - StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE, CommonConstants.Server.CONFIG_OF_SEGMENT_STORE_URI)); - } - tableDataManager = - new FailureInjectingRealtimeTableDataManager(_segmentBuildSemaphore, isServerReadyToServeQueries); - break; - default: - throw new IllegalStateException(); - } - tableDataManager.init(_instanceDataManagerConfig, _helixManager, _segmentLocks, tableConfig, segmentPreloadExecutor, - errorCache); - return tableDataManager; - } -} From 55b2b29f94c6c3a459d546b329ed209fdda07ef2 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 09:23:41 +0530 Subject: [PATCH 33/65] Revert "Remove reingestion code" This reverts commit c4b99bd85a2d5a934656451a5c00364a6fdad0a1. --- .../utils/FileUploadDownloadClient.java | 42 ++ .../PinotLLCRealtimeSegmentManager.java | 136 +++++ .../RealtimeSegmentValidationManager.java | 7 + ...timeIngestionSegmentCommitFailureTest.java | 312 ++++++++++ .../api/resources/ReIngestionResource.java | 477 +++++++++++++++ .../reingestion/ReIngestionRequest.java | 40 ++ .../reingestion/ReIngestionResponse.java | 36 ++ .../SimpleRealtimeSegmentDataManager.java | 569 ++++++++++++++++++ 8 files changed, 1619 insertions(+) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java create mode 100644 pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 7260a98155d7..02bd02002285 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -127,6 +127,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; + public static final String REINGEST_SEGMENT_PATH = "/reingestSegment"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); @@ -1276,6 +1277,47 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi httpHeaders, maxStreamRateInByte); } + /** + * Invokes the server's reIngestSegment API via a POST request with JSON payload, + * using Simple HTTP APIs. + * + * POST http://[serverURL]/reIngestSegment + * { + * "tableNameWithType": [tableName], + * "segmentName": [segmentName] + * } + */ + public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName) + throws IOException, URISyntaxException, HttpErrorStatusException { + String scheme = HTTP; + if (serverHostPort.contains(HTTPS)) { + scheme = HTTPS; + serverHostPort = serverHostPort.replace(HTTPS + "://", ""); + } else if (serverHostPort.contains(HTTP)) { + serverHostPort = serverHostPort.replace(HTTP + "://", ""); + } + + String serverHost = serverHostPort.split(":")[0]; + String serverPort = serverHostPort.split(":")[1]; + + URI reIngestUri = getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); + + Map requestJson = new HashMap<>(); + requestJson.put("tableNameWithType", tableNameWithType); + requestJson.put("segmentName", segmentName); + + String jsonPayload = JsonUtils.objectToString(requestJson); + SimpleHttpResponse response = + HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); + + // Check that we got a 2xx response + int statusCode = response.getStatusCode(); + if (statusCode / 100 != 2) { + throw new IOException(String.format("Failed POST to %s, HTTP %d: %s", + reIngestUri, statusCode, response.getResponse())); + } + } + /** * Generate a param list with a table name attribute. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 024f00afa56d..c51dda97fa37 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.io.IOException; @@ -48,8 +49,10 @@ import org.apache.helix.ClusterMessagingService; import org.apache.helix.Criteria; import org.apache.helix.HelixAdmin; +import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; +import org.apache.helix.PropertyKey; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.store.zk.ZkHelixPropertyStore; @@ -2195,6 +2198,139 @@ URI createSegmentPath(String rawTableName, String segmentName) { return URIUtils.getUri(_controllerConf.getDataDir(), rawTableName, URIUtils.encode(segmentName)); } + /** + * Re-ingests segments that are in DONE status with a missing download URL, but also + * have no peer copy on any server. This method will call the server reIngestSegment API + * on one of the alive servers that are supposed to host that segment according to IdealState. + * + * API signature: + * POST http://[serverURL]/reIngestSegment + * Request body (JSON): + * { + * "tableNameWithType": [tableName], + * "segmentName": [segmentName], + * "uploadURI": [leadControllerUrl], + * "uploadSegment": true + * } + * + * @param tableNameWithType The table name with type, e.g. "myTable_REALTIME" + */ + public void reIngestSegmentsWithErrorState(String tableNameWithType) { + // Step 1: Fetch the ExternalView and all segments + ExternalView externalView = getExternalView(tableNameWithType); + IdealState idealState = getIdealState(tableNameWithType); + Map> segmentToInstanceCurrentStateMap = externalView.getRecord().getMapFields(); + Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); + + // find segments in ERROR state in externalView + List segmentsInErrorState = new ArrayList<>(); + for (Map.Entry> entry : segmentToInstanceCurrentStateMap.entrySet()) { + String segmentName = entry.getKey(); + Map instanceStateMap = entry.getValue(); + boolean allReplicasInError = true; + for (String state : instanceStateMap.values()) { + if (!SegmentStateModel.ERROR.equals(state)) { + allReplicasInError = false; + break; + } + } + if (allReplicasInError) { + segmentsInErrorState.add(segmentName); + } + } + + if (segmentsInErrorState.isEmpty()) { + LOGGER.info("No segments found in ERROR state for table {}", tableNameWithType); + return; + } + + // filter out segments that are not ONLINE in IdealState + for (String segmentName : segmentsInErrorState) { + Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); + boolean isOnline = true; + for (String state : instanceIdealStateMap.values()) { + if (!SegmentStateModel.ONLINE.equals(state)) { + isOnline = false; + break; + } + } + if (!isOnline) { + segmentsInErrorState.remove(segmentName); + } + } + + // Step 2: For each segment, check the ZK metadata for conditions + for (String segmentName : segmentsInErrorState) { + // Skip non-LLC segments or segments missing from the ideal state altogether + LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); + if (llcSegmentName == null || !segmentToInstanceCurrentStateMap.containsKey(segmentName)) { + continue; + } + + SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); + // We only consider segments that are in COMMITTING which is indicated by having an endOffset + // but have a missing or placeholder download URL + if (segmentZKMetadata.getStatus() == Status.COMMITTING) { + Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); + + // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive + LOGGER.info( + "Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", + segmentName, tableNameWithType); + + // Find at least one server that should host this segment and is alive + String aliveServer = findAliveServerToReIngest(instanceStateMap.keySet()); + if (aliveServer == null) { + LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, tableNameWithType); + continue; + } + + try { + _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName); + LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); + } catch (Exception e) { + LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); + } + } else if (segmentZKMetadata.getStatus() == Status.UPLOADED) { + LOGGER.info( + "Segment {} in table {} is in ERROR state with download URL present. Resetting segment to ONLINE state.", + segmentName, tableNameWithType); + _helixResourceManager.resetSegment(tableNameWithType, segmentName, null); + } + } + } + + /** + * Picks one 'alive' server among a set of servers that are supposed to host the segment, + * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. + * This is a simple example; adapt to your environment’s definition of “alive.” + */ + private String findAliveServerToReIngest(Set candidateServers) { + // Get the current live instances from Helix + HelixDataAccessor helixDataAccessor = _helixManager.getHelixDataAccessor(); + PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder(); + List liveInstances = helixDataAccessor.getChildNames(keyBuilder.liveInstances()); + try { + // This should ideally handle https scheme as well + BiMap instanceToEndpointMap = + _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); + + if (instanceToEndpointMap.isEmpty()) { + LOGGER.warn("No instance data admin endpoints found for servers: {}", candidateServers); + return null; + } + + for (String server : candidateServers) { + if (liveInstances.contains(server)) { + return instanceToEndpointMap.get(server); + } + } + } catch (Exception e) { + LOGGER.warn("Failed to get Helix instance data admin endpoints for servers: {}", candidateServers, e); + } + return null; + } + @VisibleForTesting public void enableTestFault(String faultType) { _failureConfig.put(faultType, "true"); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 510b292a83bb..e4411f7ae441 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -28,6 +28,7 @@ import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.metrics.ValidationMetrics; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.LeadControllerManager; import org.apache.pinot.controller.api.resources.PauseStatusDetails; @@ -174,6 +175,12 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { // Update the total document count gauge _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); + boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); + + if (isPauselessConsumptionEnabled) { + _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); + } + // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java new file mode 100644 index 000000000000..e37714d9976c --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -0,0 +1,312 @@ +/** + * 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.integration.tests; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.io.FileUtils; +import org.apache.helix.model.ExternalView; +import org.apache.helix.model.IdealState; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.spi.stream.StreamConfigProperties.SEGMENT_COMPLETION_FSM_SCHEME; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; + + +public class PauselessRealtimeIngestionSegmentCommitFailureTest extends BaseClusterIntegrationTest { + + private static final int NUM_REALTIME_SEGMENTS = 48; + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + private static final Logger LOGGER = + LoggerFactory.getLogger(PauselessRealtimeIngestionSegmentCommitFailureTest.class); + private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private List _avroFiles; + + protected void overrideControllerConf(Map properties) { + properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); + properties.put(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT, true); + properties.put(SegmentCompletionConfig.FSM_SCHEME + "pauseless", + "org.apache.pinot.controller.helix.core.realtime.PauselessSegmentCompletionFSM"); + // Set the delay more than the time we sleep before triggering RealtimeSegmentValidationManager manually, i.e. + // MAX_SEGMENT_COMPLETION_TIME_MILLIS, to ensure that the segment level validations are performed. + properties.put(ControllerConf.ControllerPeriodicTasksConf.REALTIME_SEGMENT_VALIDATION_INITIAL_DELAY_IN_SECONDS, + 500); + } + + @Override + protected void overrideServerConf(PinotConfiguration serverConf) { + // Set segment store uri to the one used by controller as data dir (i.e. deep store) + try { + LOGGER.info("Set segment.store.uri: {} for server with scheme: {}", _controllerConfig.getDataDir(), + new URI(_controllerConfig.getDataDir()).getScheme()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + serverConf.setProperty("pinot.server.instance.segment.store.uri", "file:" + _controllerConfig.getDataDir()); + serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, + "true"); + serverConf.setProperty("pinot.server.instance." + CommonConstants.Server.TABLE_DATA_MANAGER_PROVIDER_CLASS, + "org.apache.pinot.integration.tests.realtime.FailureInjectingTableDataManagerProvider"); + } + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + // Start the Pinot cluster + startZk(); + // Start a customized controller with more frequent realtime segment validation + startController(); + startBroker(); + startServer(); + + // load data in kafka + _avroFiles = unpackAvroData(_tempDir); + startKafka(); + pushAvroIntoKafka(_avroFiles); + + // create schema for non-pauseless table + Schema schema = createSchema(); + schema.setSchemaName(DEFAULT_TABLE_NAME_2); + addSchema(schema); + + // add non-pauseless table + TableConfig tableConfig2 = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig2.setTableName(DEFAULT_TABLE_NAME_2); + tableConfig2.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig2.getValidationConfig().setRetentionTimeValue("100000"); + addTableConfig(tableConfig2); + + // Ensure that the commit protocol for all the segments have completed before injecting failure + waitForDocsLoaded(600_000L, true, tableConfig2.getTableName()); + TestUtils.waitForCondition((aVoid) -> { + List segmentZKMetadataList = + _helixResourceManager.getSegmentsZKMetadata(tableConfig2.getTableName()); + return assertUrlPresent(segmentZKMetadataList); + }, 1000, 100000, "Some segments still have missing url"); + + // create schema for pauseless table + schema.setSchemaName(DEFAULT_TABLE_NAME); + addSchema(schema); + + // add pauseless table + TableConfig tableConfig = createRealtimeTableConfig(_avroFiles.get(0)); + tableConfig.getValidationConfig().setRetentionTimeUnit("DAYS"); + tableConfig.getValidationConfig().setRetentionTimeValue("100000"); + + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setStreamIngestionConfig( + new StreamIngestionConfig(List.of(tableConfig.getIndexingConfig().getStreamConfigs()))); + ingestionConfig.getStreamIngestionConfig().setPauselessConsumptionEnabled(true); + Map streamConfigMap = ingestionConfig.getStreamIngestionConfig() + .getStreamConfigMaps() + .get(0); + streamConfigMap.put(SEGMENT_COMPLETION_FSM_SCHEME, "pauseless"); + streamConfigMap.put("segmentDownloadTimeoutMinutes", "1"); + tableConfig.getIndexingConfig().setStreamConfigs(null); + tableConfig.setIngestionConfig(ingestionConfig); + + addTableConfig(tableConfig); + Thread.sleep(60000L); + TestUtils.waitForCondition( + (aVoid) -> atLeastOneErrorSegmentInExternalView(TableNameBuilder.REALTIME.tableNameWithType(getTableName())), + 1000, 600000, "Segments still not in error state"); + } + + @Test + public void testSegmentAssignment() + throws Exception { + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); + + // 1) Capture which segments went into the ERROR state + List erroredSegments = getErroredSegmentsInExternalView(tableNameWithType); + assertFalse(erroredSegments.isEmpty(), "No segments found in ERROR state, expected at least one."); + + // Let the RealtimeSegmentValidationManager run so it can fix up segments + Thread.sleep(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + LOGGER.info("Triggering RealtimeSegmentValidationManager to reingest errored segments"); + _controllerStarter.getRealtimeSegmentValidationManager().run(); + LOGGER.info("Finished RealtimeSegmentValidationManager to reingest errored segments"); + + // Wait until there are no ERROR segments in the ExternalView + TestUtils.waitForCondition(aVoid -> { + List errorSegmentsRemaining = getErroredSegmentsInExternalView(tableNameWithType); + return errorSegmentsRemaining.isEmpty(); + }, 10000, 100000, "Some segments are still in ERROR state after resetSegments()"); + + // Finally compare metadata across your two tables + compareZKMetadataForSegments(_helixResourceManager.getSegmentsZKMetadata(tableNameWithType), + _helixResourceManager.getSegmentsZKMetadata( + TableNameBuilder.REALTIME.tableNameWithType(DEFAULT_TABLE_NAME_2))); + } + + /** + * Returns the list of segment names in ERROR state from the ExternalView of the given table. + */ + private List getErroredSegmentsInExternalView(String tableName) { + ExternalView resourceEV = _helixResourceManager.getHelixAdmin() + .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); + Map> segmentAssignment = resourceEV.getRecord().getMapFields(); + List erroredSegments = new ArrayList<>(); + for (Map.Entry> entry : segmentAssignment.entrySet()) { + String segmentName = entry.getKey(); + Map serverToStateMap = entry.getValue(); + for (String state : serverToStateMap.values()) { + if ("ERROR".equals(state)) { + erroredSegments.add(segmentName); + break; // No need to check other servers for this segment + } + } + } + return erroredSegments; + } + + /** + * Checks that all segments which were previously in ERROR state now have status == UPLOADED. + */ + private boolean haveErroredSegmentsUploaded( + List segmentZKMetadataList, List previouslyErroredSegments) { + + // Convert to a Set for quick lookups + Set erroredSegmentNames = new HashSet<>(previouslyErroredSegments); + + for (SegmentZKMetadata metadata : segmentZKMetadataList) { + if (erroredSegmentNames.contains(metadata.getSegmentName())) { + // If it was previously ERROR, then we expect it to have transitioned to UPLOADED + if (metadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { + return false; + } + } + } + return true; + } + + + private void compareZKMetadataForSegments(List segmentsZKMetadata, + List segmentsZKMetadata1) { + Map segmentZKMetadataMap = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata); + Map segmentZKMetadataMap1 = getPartitionSegmentNumberToMetadataMap(segmentsZKMetadata1); + segmentZKMetadataMap.forEach((segmentKey, segmentZKMetadata) -> { + SegmentZKMetadata segmentZKMetadata1 = segmentZKMetadataMap1.get(segmentKey); + areSegmentZkMetadataSame(segmentZKMetadata, segmentZKMetadata1); + }); + } + + private void areSegmentZkMetadataSame(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { + if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + return; + } + assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); + assertEquals(segmentZKMetadata.getStartOffset(), segmentZKMetadata1.getStartOffset()); + assertEquals(segmentZKMetadata.getEndOffset(), segmentZKMetadata1.getEndOffset()); + assertEquals(segmentZKMetadata.getTotalDocs(), segmentZKMetadata1.getTotalDocs()); + assertEquals(segmentZKMetadata.getStartTimeMs(), segmentZKMetadata1.getStartTimeMs()); + assertEquals(segmentZKMetadata.getEndTimeMs(), segmentZKMetadata1.getEndTimeMs()); + } + + private Map getPartitionSegmentNumberToMetadataMap( + List segmentsZKMetadata) { + Map segmentZKMetadataMap = new HashMap<>(); + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + String segmentKey = llcSegmentName.getPartitionGroupId() + "_" + llcSegmentName.getSequenceNumber(); + segmentZKMetadataMap.put(segmentKey, segmentZKMetadata); + } + return segmentZKMetadataMap; + } + + @AfterClass + public void tearDown() + throws IOException { + LOGGER.info("Tearing down..."); + dropRealtimeTable(getTableName()); + stopServer(); + stopBroker(); + stopController(); + stopKafka(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + private void verifyIdealState(String tableName, int numSegmentsExpected) { + IdealState idealState = HelixHelper.getTableIdealState(_helixManager, tableName); + Map> segmentAssignment = idealState.getRecord().getMapFields(); + assertEquals(segmentAssignment.size(), numSegmentsExpected); + } + + private boolean atLeastOneErrorSegmentInExternalView(String tableName) { + ExternalView resourceEV = _helixResourceManager.getHelixAdmin() + .getResourceExternalView(_helixResourceManager.getHelixClusterName(), tableName); + Map> segmentAssigment = resourceEV.getRecord().getMapFields(); + for (Map serverToStateMap : segmentAssigment.values()) { + for (String state : serverToStateMap.values()) { + if (state.equals("ERROR")) { + return true; + } + } + } + return false; + } + + private void assertUploadUrlEmpty(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + assertNull(segmentZKMetadata.getDownloadUrl()); + } + } + + private boolean assertUrlPresent(List segmentZKMetadataList) { + for (SegmentZKMetadata segmentZKMetadata : segmentZKMetadataList) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.COMMITTING + && segmentZKMetadata.getDownloadUrl() == null) { + LOGGER.warn("URl not found for segment: {}", segmentZKMetadata.getSegmentName()); + return false; + } + } + return true; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java new file mode 100644 index 000000000000..5c145beedcc0 --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -0,0 +1,477 @@ +/** + * 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.server.api.resources; + +import com.google.common.base.Function; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiKeyAuthDefinition; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Authorization; +import io.swagger.annotations.SecurityDefinition; +import io.swagger.annotations.SwaggerDefinition; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import javax.inject.Inject; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.NameValuePair; +import org.apache.hc.core5.http.message.BasicHeader; +import org.apache.hc.core5.http.message.BasicNameValuePair; +import org.apache.pinot.common.auth.AuthProviderUtils; +import org.apache.pinot.common.exception.HttpErrorStatusException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.SimpleHttpResponse; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.common.utils.URIUtils; +import org.apache.pinot.common.utils.http.HttpClient; +import org.apache.pinot.core.data.manager.InstanceDataManager; +import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.segment.local.data.manager.SegmentDataManager; +import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; +import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; +import org.apache.pinot.server.realtime.ControllerLeaderLocator; +import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; +import org.apache.pinot.server.starter.ServerInstance; +import org.apache.pinot.spi.auth.AuthProvider; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.StringUtil; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; +import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; +import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; + + +@Api(tags = "ReIngestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), + @Authorization(value = DATABASE)}) +@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = { + @ApiKeyAuthDefinition(name = HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, + key = SWAGGER_AUTHORIZATION_KEY, + description = "The format of the key is ```\"Basic \" or \"Bearer \"```"), + @ApiKeyAuthDefinition(name = DATABASE, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = DATABASE, + description = "Database context passed through http header. If no context is provided 'default' database " + + "context will be considered.")})) +@Path("/") +public class ReIngestionResource { + private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); + public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + //TODO: Maximum number of concurrent re-ingestions allowed should be configurable + private static final int MAX_PARALLEL_REINGESTIONS = 10; + + // Map to track ongoing ingestion per segment + private static final ConcurrentHashMap SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + + // Semaphore to enforce global concurrency limit + private static final Semaphore REINGESTION_SEMAPHORE = new Semaphore(MAX_PARALLEL_REINGESTIONS); + + @Inject + private ServerInstance _serverInstance; + + @POST + @Path("/reingestSegment") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " + + "upload the segment") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, + message = "Internal server error", response = ErrorInfo.class) + }) + public Response reIngestSegment(ReIngestionRequest request) { + try { + String tableNameWithType = request.getTableNameWithType(); + String segmentName = request.getSegmentName(); + + // Try to acquire a permit from the semaphore to ensure we don't exceed max concurrency + if (!REINGESTION_SEMAPHORE.tryAcquire()) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE) + .entity("Too many re-ingestions in progress. Please try again later.") + .build(); + } + + // Check if the segment is already being re-ingested + AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); + if (!isIngesting.compareAndSet(false, true)) { + // The segment is already being ingested + REINGESTION_SEMAPHORE.release(); + return Response.status(Response.Status.CONFLICT) + .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") + .build(); + } + + InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); + if (instanceDataManager == null) { + throw new WebApplicationException(new RuntimeException("Invalid server initialization"), + Response.Status.INTERNAL_SERVER_ERROR); + } + + TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); + if (tableDataManager == null) { + throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); + LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); + + // Get TableConfig and Schema + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + if (tableConfig == null) { + throw new WebApplicationException("Table config not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + Schema schema = indexLoadingConfig.getSchema(); + if (schema == null) { + throw new WebApplicationException("Schema not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + // Fetch SegmentZKMetadata + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (segmentZKMetadata == null) { + throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, + Response.Status.NOT_FOUND); + } + + // Get startOffset, endOffset, partitionGroupId + String startOffsetStr = segmentZKMetadata.getStartOffset(); + String endOffsetStr = segmentZKMetadata.getEndOffset(); + + if (startOffsetStr == null || endOffsetStr == null) { + return Response.serverError().entity("Start offset or end offset is null for segment: " + segmentName).build(); + } + + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + int partitionGroupId = llcSegmentName.getPartitionGroupId(); + + Map streamConfigMap; + try { + streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + } catch (Exception e) { + return Response.serverError().entity("Failed to get stream config for table: " + tableNameWithType).build(); + } + + StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); + + // Set up directories + File resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); + File resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); + + if (!resourceTmpDir.exists()) { + resourceTmpDir.mkdirs(); + } + if (!resourceDataDir.exists()) { + resourceDataDir.mkdirs(); + } + + LOGGER.info("Starting SimpleRealtimeSegmentDataManager..."); + // Instantiate SimpleRealtimeSegmentDataManager + SimpleRealtimeSegmentDataManager manager = + new SimpleRealtimeSegmentDataManager(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, + tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, resourceTmpDir, + resourceDataDir, _serverInstance.getServerMetrics()); + + try { + + manager.startConsumption(); + + waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300000, 0); + + manager.stopConsumption(); + + // After ingestion is complete, get the segment + if (!manager.isSuccess()) { + throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); + } + + LOGGER.info("Starting build for segment {}", segmentName); + SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + manager.buildSegmentInternal(); + + // Get the segment directory + File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + + if (segmentTarFile == null) { + throw new Exception("Failed to build segment: " + segmentName); + } + + ServerSegmentCompletionProtocolHandler protocolHandler = + new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); + + AuthProvider authProvider = protocolHandler.getAuthProvider(); + List
headers = AuthProviderUtils.toRequestHeaders(authProvider); + + String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); + + pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); + + LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); + // wait for segment metadata to have status as UPLOADED + waitForCondition((Void) -> { + SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { + return false; + } + + SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName); + return segmentDataManager instanceof ImmutableSegmentDataManager; + }, 5000, 300000, 0); + + // trigger segment reset call on API + LOGGER.info("Triggering segment reset for uploaded segment {}", segmentName); + HttpClient httpClient = HttpClient.getInstance(); + Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); + resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); + + LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); + } catch (Exception e) { + return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); + } finally { + // Clean up + manager.offload(); + manager.destroy(); + + // Delete temporary directories + FileUtils.deleteQuietly(resourceTmpDir); + FileUtils.deleteQuietly(resourceDataDir); + + isIngesting.set(false); + } + // Return success response + return Response.ok().entity(new ReIngestionResponse("Segment re-ingested and uploaded successfully")).build(); + } catch (Exception e) { + LOGGER.error("Error during re-ingestion", e); + throw new WebApplicationException(e, Response.Status.INTERNAL_SERVER_ERROR); + } finally { + REINGESTION_SEMAPHORE.release(); + } + } + + private void waitForCondition( + Function condition, long checkIntervalMs, long timeoutMs, long gracePeriodMs) { + long endTime = System.currentTimeMillis() + timeoutMs; + + // Adding grace period before starting the condition checks + if (gracePeriodMs > 0) { + LOGGER.info("Waiting for a grace period of {} ms before starting condition checks", gracePeriodMs); + try { + Thread.sleep(gracePeriodMs); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted during grace period wait", e); + } + } + + while (System.currentTimeMillis() < endTime) { + try { + if (Boolean.TRUE.equals(condition.apply(null))) { + LOGGER.info("Condition satisfied: {}", condition); + return; + } + Thread.sleep(checkIntervalMs); + } catch (Exception e) { + throw new RuntimeException("Caught exception while checking the condition", e); + } + } + + throw new RuntimeException("Timeout waiting for condition: " + condition); + } + + public void resetSegment(HttpClient httpClient, String controllerVipUrl, String tableNameWithType, String segmentName, + String targetInstance, Map headers) + throws IOException { + try { + //TODO: send correct headers + HttpClient.wrapAndThrowHttpException(httpClient.sendJsonPostRequest( + new URI(getURLForSegmentReset(controllerVipUrl, tableNameWithType, segmentName, targetInstance)), null, + headers)); + } catch (HttpErrorStatusException | URISyntaxException e) { + throw new IOException(e); + } + } + + private String getURLForSegmentReset(String controllerVipUrl, String tableNameWithType, String segmentName, + @Nullable String targetInstance) { + String query = targetInstance == null ? "reset" : "reset?targetInstance=" + targetInstance; + return StringUtil.join("/", controllerVipUrl, "segments", tableNameWithType, encode(segmentName), query); + } + + private String encode(String s) { + return URLEncoder.encode(s, StandardCharsets.UTF_8); + } + + /** + * Push segment metadata to the Pinot Controller in METADATA mode. + * + * @param tableNameWithType The table name with type (e.g., "myTable_OFFLINE") + * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") + * @param segmentFile The local segment tar.gz file + * @param authHeaders A map of authentication or additional headers for the request + */ + public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, + List
authHeaders, String segmentName, ServerSegmentCompletionProtocolHandler protocolHandler) + throws Exception { + LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), + tableNameWithType, controllerUrl); + String tableName = tableNameWithType; + File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); + + LOGGER.info("Generated segment metadata tar file: {}", segmentMetadataFile.getAbsolutePath()); + try { + // Prepare headers + List
headers = authHeaders; + + // The upload type must be METADATA + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, + FileUploadDownloadClient.FileUploadType.METADATA.toString())); + + // The DOWNLOAD_URI header specifies where the controller can fetch the segment if needed + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentFile.toURI().toString())); + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); + + // Set table name parameter + List parameters = getSegmentPushCommonParams(tableNameWithType); + + // Construct the endpoint URI + URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); + + LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); + + // Perform the metadata upload + SimpleHttpResponse response = protocolHandler.getFileUploadDownloadClient() + .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, + HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + + LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, tableName, + controllerUrl, response.getStatusCode(), response.getResponse()); + } finally { + FileUtils.deleteQuietly(segmentMetadataFile); + } + } + + private List getSegmentPushCommonParams(String tableNameWithType) { + List params = new ArrayList<>(); + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION, + "true")); + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_NAME, + TableNameBuilder.extractRawTableName(tableNameWithType))); + TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); + if (tableType != null) { + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_TYPE, tableType.toString())); + } else { + throw new RuntimeException(String.format("Failed to determine the tableType from name: %s", tableNameWithType)); + } + return params; + } + + /** + * Generate a tar.gz file containing only the metadata files (metadata.properties, creation.meta) + * from a given Pinot segment tar.gz file. + */ + private File generateSegmentMetadataTar(File segmentTarFile) + throws Exception { + + if (!segmentTarFile.exists()) { + throw new IllegalArgumentException("Segment tar file does not exist: " + segmentTarFile.getAbsolutePath()); + } + + LOGGER.info("Generating segment metadata tar file from segment tar: {}", segmentTarFile.getAbsolutePath()); + File tempDir = Files.createTempDirectory("pinot-segment-temp").toFile(); + String uuid = UUID.randomUUID().toString(); + try { + File metadataDir = new File(tempDir, "segmentMetadataDir-" + uuid); + if (!metadataDir.mkdirs()) { + throw new RuntimeException("Failed to create metadata directory: " + metadataDir.getAbsolutePath()); + } + + LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, + new File(metadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); + + // Extract creation.meta + LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.SEGMENT_CREATION_META, + new File(metadataDir, V1Constants.SEGMENT_CREATION_META)); + + File segmentMetadataFile = + new File(FileUtils.getTempDirectory(), "segmentMetadata-" + UUID.randomUUID() + ".tar.gz"); + TarCompressionUtils.createCompressedTarFile(metadataDir, segmentMetadataFile); + return segmentMetadataFile; + } finally { + FileUtils.deleteQuietly(tempDir); + } + } + + private String getControllerUrl(String rawTableName, ServerSegmentCompletionProtocolHandler protocolHandler) { + ControllerLeaderLocator leaderLocator = ControllerLeaderLocator.getInstance(); + final Pair leaderHostPort = leaderLocator.getControllerLeader(rawTableName); + if (leaderHostPort == null) { + LOGGER.warn("No leader found for table: {}", rawTableName); + return null; + } + Integer port = leaderHostPort.getRight(); + String protocol = protocolHandler.getProtocol(); + Integer controllerHttpsPort = protocolHandler.getControllerHttpsPort(); + if (controllerHttpsPort != null) { + port = controllerHttpsPort; + protocol = HTTPS_PROTOCOL; + } + + return URIUtils.buildURI(protocol, leaderHostPort.getLeft() + ":" + port, "", Collections.emptyMap()).toString(); + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java new file mode 100644 index 000000000000..a2b92379e549 --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java @@ -0,0 +1,40 @@ +/** + * 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.server.api.resources.reingestion; + +public class ReIngestionRequest { + private String _tableNameWithType; + private String _segmentName; + + public String getTableNameWithType() { + return _tableNameWithType; + } + + public void setTableNameWithType(String tableNameWithType) { + _tableNameWithType = tableNameWithType; + } + + public String getSegmentName() { + return _segmentName; + } + + public void setSegmentName(String segmentName) { + _segmentName = segmentName; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java new file mode 100644 index 000000000000..af1b5f7d55bb --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java @@ -0,0 +1,36 @@ +/** + * 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.server.api.resources.reingestion; + +public class ReIngestionResponse { + private String _message; + + public ReIngestionResponse(String message) { + _message = message; + } + + // Getter and setter + public String getMessage() { + return _message; + } + + public void setMessage(String message) { + _message = message; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java new file mode 100644 index 000000000000..23b4b49e111c --- /dev/null +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -0,0 +1,569 @@ +/** + * 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.server.api.resources.reingestion.utils; + +import com.google.common.annotations.VisibleForTesting; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.segment.local.data.manager.SegmentDataManager; +import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; +import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager; +import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; +import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; +import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; +import org.apache.pinot.segment.local.segment.creator.TransformPipeline; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.utils.IngestionUtils; +import org.apache.pinot.segment.spi.MutableSegment; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; +import org.apache.pinot.spi.config.table.SegmentPartitionConfig; +import org.apache.pinot.spi.config.table.SegmentZKPropsConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.plugin.PluginManager; +import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.StreamConsumerFactory; +import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; +import org.apache.pinot.spi.stream.StreamDataDecoder; +import org.apache.pinot.spi.stream.StreamDataDecoderImpl; +import org.apache.pinot.spi.stream.StreamDataDecoderResult; +import org.apache.pinot.spi.stream.StreamMessage; +import org.apache.pinot.spi.stream.StreamMessageDecoder; +import org.apache.pinot.spi.stream.StreamMetadataProvider; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; +import org.apache.pinot.spi.utils.retry.RetryPolicies; +import org.apache.pinot.spi.utils.retry.RetryPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Simplified Segment Data Manager for ingesting data from a start offset to an end offset. + */ +public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { + + private static final int DEFAULT_CAPACITY = 100_000; + private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; + public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + + private final String _segmentName; + private final String _tableNameWithType; + private final int _partitionGroupId; + private final String _segmentNameStr; + private final SegmentZKMetadata _segmentZKMetadata; + private final TableConfig _tableConfig; + private final Schema _schema; + private final StreamConfig _streamConfig; + private final StreamPartitionMsgOffsetFactory _offsetFactory; + private final StreamConsumerFactory _consumerFactory; + private StreamMetadataProvider _partitionMetadataProvider; + private final PartitionGroupConsumer _consumer; + private final StreamDataDecoder _decoder; + private final MutableSegmentImpl _realtimeSegment; + private final File _resourceTmpDir; + private final File _resourceDataDir; + private final Logger _logger; + private Thread _consumerThread; + private final AtomicBoolean _shouldStop = new AtomicBoolean(false); + private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false); + private final StreamPartitionMsgOffset _startOffset; + private final StreamPartitionMsgOffset _endOffset; + private volatile StreamPartitionMsgOffset _currentOffset; + private volatile int _numRowsIndexed = 0; + private final String _segmentStoreUriStr; + private final int _fetchTimeoutMs; + private final TransformPipeline _transformPipeline; + private volatile boolean _isSuccess = false; + private volatile Throwable _consumptionException; + private final ServerMetrics _serverMetrics; + + public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, + SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, + IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, + File resourceTmpDir, File resourceDataDir, ServerMetrics serverMetrics) + throws Exception { + + _segmentName = segmentName; + _tableNameWithType = tableNameWithType; + _partitionGroupId = partitionGroupId; + _segmentZKMetadata = segmentZKMetadata; + _tableConfig = tableConfig; + _schema = schema; + _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI(); + _streamConfig = streamConfig; + _resourceTmpDir = resourceTmpDir; + _resourceDataDir = resourceDataDir; + _serverMetrics = serverMetrics; + _logger = LoggerFactory.getLogger(SimpleRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); + + _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); + _startOffset = _offsetFactory.create(startOffsetStr); + _endOffset = _offsetFactory.create(endOffsetStr); + + String clientId = getClientId(); + + _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig); + _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId); + _segmentNameStr = _segmentZKMetadata.getSegmentName(); + + // Create a simple PartitionGroupConsumptionStatus + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = + new PartitionGroupConsumptionStatus(_partitionGroupId, 0, _startOffset, null, null); + + _consumer = _consumerFactory.createPartitionGroupConsumer(clientId, partitionGroupConsumptionStatus); + + // Initialize decoder + Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig, _schema); + _decoder = createDecoder(fieldsToRead); + + // Fetch capacity from indexLoadingConfig or use default + int capacity = streamConfig.getFlushThresholdRows(); + if (capacity <= 0) { + capacity = DEFAULT_CAPACITY; + } + + // Fetch average number of multi-values from indexLoadingConfig + int avgNumMultiValues = indexLoadingConfig.getRealtimeAvgMultiValueCount(); + + // Load stats history, here we are using the same stats while as the RealtimeSegmentDataManager so that we are + // much more efficient in allocating buffers. It also works with empty file + String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != null + ? indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + File.separator + _tableNameWithType + : resourceTmpDir.getAbsolutePath(); + File statsHistoryFile = new File(tableDataDir, "segment-stats.ser"); + RealtimeSegmentStatsHistory statsHistory = RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile); + + // Initialize mutable segment with configurations + RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = + new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType) + .setSegmentName(_segmentName).setStreamName(_streamConfig.getTopicName()) + .setSegmentZKMetadata(_segmentZKMetadata).setStatsHistory(statsHistory).setSchema(_schema) + .setCapacity(capacity).setAvgNumMultiValues(avgNumMultiValues) + .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) + .setFieldConfigList(tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) + .setMemoryManager( + new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); + + setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); + + _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), _serverMetrics); + + _transformPipeline = new TransformPipeline(tableConfig, schema); + + // Initialize fetch timeout + _fetchTimeoutMs = + _streamConfig.getFetchTimeoutMillis() > 0 ? _streamConfig.getFetchTimeoutMillis() : DEFAULT_FETCH_TIMEOUT_MS; + } + + private String getClientId() { + return _tableNameWithType + "-" + _partitionGroupId; + } + + public void startConsumption() { + // Start the consumer thread + _consumerThread = new Thread(new PartitionConsumer(), _segmentName); + _consumerThread.start(); + } + + private StreamDataDecoder createDecoder(Set fieldsToRead) + throws Exception { + AtomicReference localStreamDataDecoder = new AtomicReference<>(); + RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetryPolicy(5, 1000L, 1.2f); + retryPolicy.attempt(() -> { + try { + StreamMessageDecoder streamMessageDecoder = createMessageDecoder(fieldsToRead); + localStreamDataDecoder.set(new StreamDataDecoderImpl(streamMessageDecoder)); + return true; + } catch (Exception e) { + _logger.warn("Failed to create StreamMessageDecoder. Retrying...", e); + return false; + } + }); + return localStreamDataDecoder.get(); + } + + /** + * Creates a {@link StreamMessageDecoder} using properties in {@link StreamConfig}. + * + * @param fieldsToRead The fields to read from the source stream + * @return The initialized StreamMessageDecoder + */ + private StreamMessageDecoder createMessageDecoder(Set fieldsToRead) { + String decoderClass = _streamConfig.getDecoderClass(); + try { + StreamMessageDecoder decoder = PluginManager.get().createInstance(decoderClass); + decoder.init(fieldsToRead, _streamConfig, _tableConfig, _schema); + return decoder; + } catch (Exception e) { + throw new RuntimeException( + "Caught exception while creating StreamMessageDecoder from stream config: " + _streamConfig, e); + } + } + + private class PartitionConsumer implements Runnable { + @Override + public void run() { + try { + _consumer.start(_startOffset); + _currentOffset = _startOffset; + TransformPipeline.Result reusedResult = new TransformPipeline.Result(); + while (!_shouldStop.get() && _currentOffset.compareTo(_endOffset) < 0) { + // Fetch messages + MessageBatch messageBatch = _consumer.fetchMessages(_currentOffset, _fetchTimeoutMs); + + int messageCount = messageBatch.getMessageCount(); + + for (int i = 0; i < messageCount; i++) { + if (_shouldStop.get()) { + break; + } + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + if (streamMessage.getMetadata() != null && streamMessage.getMetadata().getOffset() != null + && streamMessage.getMetadata().getOffset().compareTo(_endOffset) >= 0) { + _shouldStop.set(true); + _logger.info("Reached end offset: {} for partition group: {}", _endOffset, _partitionGroupId); + break; + } + + // Decode message + StreamDataDecoderResult decodedResult = _decoder.decode(streamMessage); + if (decodedResult.getException() == null) { + // Index message + GenericRow row = decodedResult.getResult(); + + _transformPipeline.processRow(row, reusedResult); + + List transformedRows = reusedResult.getTransformedRows(); + + // TODO: Do enrichment and transforms before indexing + for (GenericRow transformedRow : transformedRows) { + _realtimeSegment.index(transformedRow, streamMessage.getMetadata()); + _numRowsIndexed++; + } + } else { + _logger.warn("Failed to decode message at offset {}: {}", _currentOffset, decodedResult.getException()); + } + } + + _currentOffset = messageBatch.getOffsetOfNextBatch(); + } + _isSuccess = true; + } catch (Exception e) { + _logger.error("Exception in consumer thread", e); + _consumptionException = e; + throw new RuntimeException(e); + } finally { + try { + _consumer.close(); + } catch (Exception e) { + _logger.warn("Failed to close consumer", e); + } + _isDoneConsuming.set(true); + } + } + } + + public void stopConsumption() { + _shouldStop.set(true); + if (_consumerThread.isAlive()) { + _consumerThread.interrupt(); + try { + _consumerThread.join(); + } catch (InterruptedException e) { + _logger.warn("Interrupted while waiting for consumer thread to finish"); + } + } + } + + @Override + public MutableSegment getSegment() { + return _realtimeSegment; + } + + @Override + public String getSegmentName() { + return _segmentName; + } + + @Override + protected void doDestroy() { + _realtimeSegment.destroy(); + } + + @Override + public void doOffload() { + stopConsumption(); + } + + public boolean isDoneConsuming() { + return _isDoneConsuming.get(); + } + + public boolean isSuccess() { + return _isSuccess; + } + + public Throwable getConsumptionException() { + return _consumptionException; + } + + @VisibleForTesting + public SegmentBuildDescriptor buildSegmentInternal() throws Exception { + _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); + final long lockAcquireTimeMillis = now(); + // Build a segment from in-memory rows. + // Use a temporary directory + Path tempSegmentFolder = null; + try { + tempSegmentFolder = + java.nio.file.Files.createTempDirectory(_resourceTmpDir.toPath(), "tmp-" + _segmentNameStr + "-"); + } catch (IOException e) { + _logger.error("Failed to create temporary directory for segment build", e); + return null; + } + + SegmentZKPropsConfig segmentZKPropsConfig = new SegmentZKPropsConfig(); + segmentZKPropsConfig.setStartOffset(_startOffset.toString()); + segmentZKPropsConfig.setEndOffset(_endOffset.toString()); + + // Build the segment + RealtimeSegmentConverter converter = + new RealtimeSegmentConverter(_realtimeSegment, segmentZKPropsConfig, tempSegmentFolder.toString(), + _schema, _tableNameWithType, _tableConfig, _segmentZKMetadata.getSegmentName(), + _tableConfig.getIndexingConfig().isNullHandlingEnabled()); + try { + converter.build(null, _serverMetrics); + } catch (Exception e) { + _logger.error("Failed to build segment", e); + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + return null; + } + final long buildTimeMillis = now() - lockAcquireTimeMillis; + + File dataDir = _resourceDataDir; + File indexDir = new File(dataDir, _segmentNameStr); + FileUtils.deleteQuietly(indexDir); + + File tempIndexDir = new File(tempSegmentFolder.toFile(), _segmentNameStr); + if (!tempIndexDir.exists()) { + _logger.error("Temp index directory {} does not exist", tempIndexDir); + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + return null; + } + try { + FileUtils.moveDirectory(tempIndexDir, indexDir); + } catch (IOException e) { + _logger.error("Caught exception while moving index directory from: {} to: {}", tempIndexDir, indexDir, e); + return null; + } finally { + FileUtils.deleteQuietly(tempSegmentFolder.toFile()); + } + + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); + + long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); + File segmentTarFile = new File(dataDir, _segmentNameStr + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); + try { + TarCompressionUtils.createCompressedTarFile(indexDir, segmentTarFile); + } catch (IOException e) { + _logger.error("Caught exception while tarring index directory from: {} to: {}", indexDir, segmentTarFile, e); + return null; + } + + File metadataFile = SegmentDirectoryPaths.findMetadataFile(indexDir); + if (metadataFile == null) { + _logger.error("Failed to find metadata file under index directory: {}", indexDir); + return null; + } + File creationMetaFile = SegmentDirectoryPaths.findCreationMetaFile(indexDir); + if (creationMetaFile == null) { + _logger.error("Failed to find creation meta file under index directory: {}", indexDir); + return null; + } + Map metadataFiles = new HashMap<>(); + metadataFiles.put(V1Constants.MetadataKeys.METADATA_FILE_NAME, metadataFile); + metadataFiles.put(V1Constants.SEGMENT_CREATION_META, creationMetaFile); + return new SegmentBuildDescriptor(segmentTarFile, metadataFiles, _currentOffset, buildTimeMillis, buildTimeMillis, + segmentSizeBytes, segmentMetadata); + } + + protected long now() { + return System.currentTimeMillis(); + } + + public void removeSegmentFile(SegmentBuildDescriptor segmentBuildDescriptor) { + if (segmentBuildDescriptor != null) { + segmentBuildDescriptor.deleteSegmentFile(); + } + } + + /* + * set the following partition parameters in RT segment config builder: + * - partition column + * - partition function + * - partition group id + */ + private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder, + SegmentPartitionConfig segmentPartitionConfig) { + if (segmentPartitionConfig != null) { + Map columnPartitionMap = segmentPartitionConfig.getColumnPartitionMap(); + if (columnPartitionMap.size() == 1) { + Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); + String partitionColumn = entry.getKey(); + ColumnPartitionConfig columnPartitionConfig = entry.getValue(); + String partitionFunctionName = columnPartitionConfig.getFunctionName(); + + // NOTE: Here we compare the number of partitions from the config and the stream, and log a warning and emit a + // metric when they don't match, but use the one from the stream. The mismatch could happen when the + // stream partitions are changed, but the table config has not been updated to reflect the change. + // In such case, picking the number of partitions from the stream can keep the segment properly + // partitioned as long as the partition function is not changed. + int numPartitions = columnPartitionConfig.getNumPartitions(); + try { + // TODO: currentPartitionGroupConsumptionStatus should be fetched from idealState + segmentZkMetadata, + // so that we get back accurate partitionGroups info + // However this is not an issue for Kafka, since partitionGroups never expire and every partitionGroup has + // a single partition + // Fix this before opening support for partitioning in Kinesis + int numPartitionGroups = + _partitionMetadataProvider.computePartitionGroupMetadata(getClientId(), _streamConfig, + Collections.emptyList(), /*maxWaitTimeMs=*/5000).size(); + + if (numPartitionGroups != numPartitions) { + _logger.info( + "Number of stream partitions: {} does not match number of partitions in the partition config: {}, " + + "using number of stream " + "partitions", numPartitionGroups, numPartitions); + numPartitions = numPartitionGroups; + } + } catch (Exception e) { + _logger.warn("Failed to get number of stream partitions in 5s, " + + "using number of partitions in the partition config: {}", numPartitions, e); + createPartitionMetadataProvider("Timeout getting number of stream partitions"); + } + + realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); + realtimeSegmentConfigBuilder.setPartitionFunction( + PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, null)); + realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); + } else { + _logger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); + } + } + } + + /** + * Creates a new stream metadata provider + */ + private void createPartitionMetadataProvider(String reason) { + closePartitionMetadataProvider(); + _logger.info("Creating new partition metadata provider, reason: {}", reason); + _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(getClientId(), _partitionGroupId); + } + + private void closePartitionMetadataProvider() { + if (_partitionMetadataProvider != null) { + try { + _partitionMetadataProvider.close(); + } catch (Exception e) { + _logger.warn("Could not close stream metadata provider", e); + } + } + } + + public class SegmentBuildDescriptor { + final File _segmentTarFile; + final Map _metadataFileMap; + final StreamPartitionMsgOffset _offset; + final long _waitTimeMillis; + final long _buildTimeMillis; + final long _segmentSizeBytes; + final SegmentMetadataImpl _segmentMetadata; + + public SegmentBuildDescriptor(@Nullable File segmentTarFile, @Nullable Map metadataFileMap, + StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, + SegmentMetadataImpl segmentMetadata) { + _segmentTarFile = segmentTarFile; + _metadataFileMap = metadataFileMap; + _offset = _offsetFactory.create(offset); + _buildTimeMillis = buildTimeMillis; + _waitTimeMillis = waitTimeMillis; + _segmentSizeBytes = segmentSizeBytes; + _segmentMetadata = segmentMetadata; + } + + public StreamPartitionMsgOffset getOffset() { + return _offset; + } + + public long getBuildTimeMillis() { + return _buildTimeMillis; + } + + public long getWaitTimeMillis() { + return _waitTimeMillis; + } + + @Nullable + public File getSegmentTarFile() { + return _segmentTarFile; + } + + @Nullable + public Map getMetadataFiles() { + return _metadataFileMap; + } + + public long getSegmentSizeBytes() { + return _segmentSizeBytes; + } + + public void deleteSegmentFile() { + if (_segmentTarFile != null) { + FileUtils.deleteQuietly(_segmentTarFile); + } + } + + public SegmentMetadataImpl getSegmentMetadata() { + return _segmentMetadata; + } + } +} From f74df66deadc276124de0b710f0fafbeafb340cb Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 09:23:53 +0530 Subject: [PATCH 34/65] Revert "Removing files related to reingestion tests" This reverts commit 791ac210b0e1c503160cb0542af6b42e0fb8a7cc. --- ...reInjectingRealtimeSegmentDataManager.java | 66 +++++++++++++ ...lureInjectingRealtimeTableDataManager.java | 69 ++++++++++++++ ...lureInjectingTableDataManagerProvider.java | 93 +++++++++++++++++++ 3 files changed, 228 insertions(+) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java new file mode 100644 index 000000000000..072cc92e4923 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java @@ -0,0 +1,66 @@ +/** + * 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.integration.tests.realtime; + +import java.util.concurrent.Semaphore; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; + + +/** + * A specialized RealtimeSegmentDataManager that lets us inject a forced failure + * in the commit step, which occurs strictly after the segmentConsumed message. + */ +public class FailureInjectingRealtimeSegmentDataManager extends RealtimeSegmentDataManager { + + // This flag controls whether commit should forcibly fail. + private final boolean _failCommit; + + /** + * Creates a manager that will forcibly fail the commit segment step. + */ + public FailureInjectingRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, + TableConfig tableConfig, RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, + IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, + Semaphore partitionGroupConsumerSemaphore, ServerMetrics serverMetrics, + boolean failCommit) throws AttemptsExceededException, RetriableOperationException { + // Pass through to the real parent constructor + super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, + indexLoadingConfig, schema, llcSegmentName, partitionGroupConsumerSemaphore, serverMetrics, + null /* no PartitionUpsertMetadataManager */, null /* no PartitionDedupMetadataManager */, + () -> true /* isReadyToConsumeData always true for tests */); + + _failCommit = failCommit; + } + + protected SegmentBuildDescriptor buildSegmentInternal(boolean forCommit) { + if (_failCommit) { + throw new RuntimeException("Forced failure in buildSegmentInternal"); + } + return super.buildSegmentInternal(forCommit); + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java new file mode 100644 index 000000000000..7dfd703bd98a --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java @@ -0,0 +1,69 @@ +/** + * 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.integration.tests.realtime; + +import java.util.concurrent.Semaphore; +import java.util.function.BooleanSupplier; +import java.util.function.Supplier; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; +import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; +import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.upsert.PartitionUpsertMetadataManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; + + +public class FailureInjectingRealtimeTableDataManager extends RealtimeTableDataManager { + private volatile boolean _hasFailedOnce = false; + + public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore) { + this(segmentBuildSemaphore, () -> true); + } + + public FailureInjectingRealtimeTableDataManager(Semaphore segmentBuildSemaphore, + Supplier isServerReadyToServeQueries) { + super(segmentBuildSemaphore, isServerReadyToServeQueries); + } + + @Override + protected RealtimeSegmentDataManager createRealtimeSegmentDataManager(SegmentZKMetadata zkMetadata, + TableConfig tableConfig, IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, + Semaphore semaphore, PartitionUpsertMetadataManager partitionUpsertMetadataManager, + PartitionDedupMetadataManager partitionDedupMetadataManager, BooleanSupplier isTableReadyToConsumeData) + throws AttemptsExceededException, RetriableOperationException { + + boolean addFailureToCommits = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); + + if (addFailureToCommits) { + if (_hasFailedOnce) { + addFailureToCommits = false; + } else { + _hasFailedOnce = true; + } + } + return new FailureInjectingRealtimeSegmentDataManager(zkMetadata, tableConfig, this, _indexDir.getAbsolutePath(), + indexLoadingConfig, schema, llcSegmentName, semaphore, _serverMetrics, addFailureToCommits); + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java new file mode 100644 index 000000000000..0c323320d51a --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java @@ -0,0 +1,93 @@ +/** + * 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.integration.tests.realtime; + +import com.google.common.cache.Cache; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.helix.HelixManager; +import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; +import org.apache.pinot.core.data.manager.offline.DimensionTableDataManager; +import org.apache.pinot.core.data.manager.offline.OfflineTableDataManager; +import org.apache.pinot.core.data.manager.provider.TableDataManagerProvider; +import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.utils.SegmentLocks; +import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfigProperties; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; + + +/** + * Default implementation of {@link TableDataManagerProvider}. + */ +public class FailureInjectingTableDataManagerProvider implements TableDataManagerProvider { + private InstanceDataManagerConfig _instanceDataManagerConfig; + private HelixManager _helixManager; + private SegmentLocks _segmentLocks; + private Semaphore _segmentBuildSemaphore; + + @Override + public void init(InstanceDataManagerConfig instanceDataManagerConfig, HelixManager helixManager, + SegmentLocks segmentLocks) { + _instanceDataManagerConfig = instanceDataManagerConfig; + _helixManager = helixManager; + _segmentLocks = segmentLocks; + int maxParallelSegmentBuilds = instanceDataManagerConfig.getMaxParallelSegmentBuilds(); + _segmentBuildSemaphore = maxParallelSegmentBuilds > 0 ? new Semaphore(maxParallelSegmentBuilds, true) : null; + } + + @Override + public TableDataManager getTableDataManager(TableConfig tableConfig, @Nullable ExecutorService segmentPreloadExecutor, + @Nullable Cache, SegmentErrorInfo> errorCache, + Supplier isServerReadyToServeQueries) { + TableDataManager tableDataManager; + switch (tableConfig.getTableType()) { + case OFFLINE: + if (tableConfig.isDimTable()) { + tableDataManager = DimensionTableDataManager.createInstanceByTableName(tableConfig.getTableName()); + } else { + tableDataManager = new OfflineTableDataManager(); + } + break; + case REALTIME: + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + if (Boolean.parseBoolean(streamConfigMap.get(StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE)) + && StringUtils.isEmpty(_instanceDataManagerConfig.getSegmentStoreUri())) { + throw new IllegalStateException(String.format("Table has enabled %s config. But the server has not " + + "configured the segmentstore uri. Configure the server config %s", + StreamConfigProperties.SERVER_UPLOAD_TO_DEEPSTORE, CommonConstants.Server.CONFIG_OF_SEGMENT_STORE_URI)); + } + tableDataManager = + new FailureInjectingRealtimeTableDataManager(_segmentBuildSemaphore, isServerReadyToServeQueries); + break; + default: + throw new IllegalStateException(); + } + tableDataManager.init(_instanceDataManagerConfig, _helixManager, _segmentLocks, tableConfig, segmentPreloadExecutor, + errorCache); + return tableDataManager; + } +} From 1f4db11186d13d5c76a7438b043b6686bda73e63 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 10:55:28 +0530 Subject: [PATCH 35/65] Fix reingestion issue where consuming segemnts are not replaced --- .../data/manager/BaseTableDataManager.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index c1462ec5b9a5..7a6ce16ab6c1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -56,6 +56,7 @@ import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; +import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; @@ -384,15 +385,21 @@ protected void replaceSegmentIfCrcMismatch(SegmentDataManager segmentDataManager IndexLoadingConfig indexLoadingConfig) throws Exception { String segmentName = segmentDataManager.getSegmentName(); - Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, - "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); - SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); - if (hasSameCRC(zkMetadata, localMetadata)) { - _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); - return; + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + // For pauseless tables, we should replace the segment if download url is missing even if crc is same + // Without this the reingestion of ERROR segments in pauseless tables fails + // as the segment data manager is still an instance of RealtimeSegmentDataManager + if (!PauselessConsumptionUtils.isPauselessEnabled(tableConfig)) { + Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, + "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); + SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); + if (hasSameCRC(zkMetadata, localMetadata)) { + _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); + return; + } + _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, + localMetadata.getCrc(), zkMetadata.getCrc()); } - _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, - localMetadata.getCrc(), zkMetadata.getCrc()); downloadAndLoadSegment(zkMetadata, indexLoadingConfig); _logger.info("Replaced segment: {} with new CRC: {}", segmentName, zkMetadata.getCrc()); } From 8e9249c38bb92ec5f02de42a3cb2ebbcbc3affa4 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 16:18:58 +0530 Subject: [PATCH 36/65] Copy full segment to deep store before triggerring metadata upload --- .../api/resources/ReIngestionResource.java | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 5c145beedcc0..ac4f7d7ecd62 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -70,6 +70,7 @@ import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; @@ -84,6 +85,8 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.IngestionConfigUtils; @@ -259,6 +262,22 @@ public Response reIngestSegment(ReIngestionRequest request) { String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); + String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); + String segmentStoreUri = indexLoadingConfig.getSegmentStoreURI(); + String destUriStr = StringUtil.join(File.separator, segmentStoreUri, rawTableName, + SegmentCompletionUtils.generateTmpSegmentFileName(segmentName)); + try (PinotFS pinotFS = PinotFSFactory.create(new URI(segmentStoreUri).getScheme())) { + // copy segment to deep store + URI destUri = new URI(destUriStr); + if (pinotFS.exists(destUri)) { + pinotFS.delete(destUri, true); + } + pinotFS.copyFromLocalFile(segmentTarFile, destUri); + } catch (Exception e) { + throw new IOException("Failed to copy segment to deep store: " + destUriStr, e); + } + + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); @@ -380,8 +399,6 @@ public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, FileUploadDownloadClient.FileUploadType.METADATA.toString())); - // The DOWNLOAD_URI header specifies where the controller can fetch the segment if needed - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentFile.toURI().toString())); headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); // Set table name parameter From b804a69a0caea4f87941104160fcf9a393bb492f Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 20:56:32 +0530 Subject: [PATCH 37/65] Refactoring: added support for tracking running reingestion jobs --- .../api/resources/ReIngestionResource.java | 358 ++++++++++-------- .../SimpleRealtimeSegmentDataManager.java | 14 +- 2 files changed, 204 insertions(+), 168 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index ac4f7d7ecd62..9b8b6600f466 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -40,12 +40,14 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Semaphore; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.inject.Inject; import javax.ws.rs.Consumes; +import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; @@ -112,213 +114,242 @@ @Path("/") public class ReIngestionResource { private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); - public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); - //TODO: Maximum number of concurrent re-ingestions allowed should be configurable - private static final int MAX_PARALLEL_REINGESTIONS = 10; - // Map to track ongoing ingestion per segment - private static final ConcurrentHashMap SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + //TODO: Make this configurable + private static final int MAX_PARALLEL_REINGESTIONS = 8; - // Semaphore to enforce global concurrency limit - private static final Semaphore REINGESTION_SEMAPHORE = new Semaphore(MAX_PARALLEL_REINGESTIONS); + // Tracks if a particular segment is currently being re-ingested + private static final ConcurrentHashMap + SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + + // Executor for asynchronous re-ingestion + private static final ExecutorService REINGESTION_EXECUTOR = + Executors.newFixedThreadPool(MAX_PARALLEL_REINGESTIONS); + + // Keep track of jobs by jobId => job info + private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); @Inject private ServerInstance _serverInstance; + /** + * Simple data class to hold job details. + */ + private static class ReIngestionJob { + private final String _jobId; + private final String _tableNameWithType; + private final String _segmentName; + private final long _startTimeMs; + + ReIngestionJob(String jobId, String tableNameWithType, String segmentName) { + _jobId = jobId; + _tableNameWithType = tableNameWithType; + _segmentName = segmentName; + _startTimeMs = System.currentTimeMillis(); + } + + public String getJobId() { + return _jobId; + } + + public String getTableNameWithType() { + return _tableNameWithType; + } + + public String getSegmentName() { + return _segmentName; + } + + public long getStartTimeMs() { + return _startTimeMs; + } + } + + /** + * New API to get all running re-ingestion jobs. + */ + @GET + @Path("/reingestSegment/jobs") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation("Get all running re-ingestion jobs along with job IDs") + public Response getAllRunningReingestionJobs() { + // Filter only the jobs still marked as running + List runningJobs = new ArrayList<>(RUNNING_JOBS.values()); + return Response.ok(runningJobs).build(); + } + @POST @Path("/reingestSegment") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " - + "upload the segment") + @ApiOperation(value = "Re-ingest segment asynchronously", notes = "Returns a jobId immediately; ingestion runs in " + + "background.") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, - message = "Internal server error", response = ErrorInfo.class) + @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), + @ApiResponse(code = 500, message = "Internal server error", response = ErrorInfo.class) }) public Response reIngestSegment(ReIngestionRequest request) { - try { - String tableNameWithType = request.getTableNameWithType(); - String segmentName = request.getSegmentName(); - - // Try to acquire a permit from the semaphore to ensure we don't exceed max concurrency - if (!REINGESTION_SEMAPHORE.tryAcquire()) { - return Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity("Too many re-ingestions in progress. Please try again later.") - .build(); - } + String tableNameWithType = request.getTableNameWithType(); + String segmentName = request.getSegmentName(); - // Check if the segment is already being re-ingested - AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); - if (!isIngesting.compareAndSet(false, true)) { - // The segment is already being ingested - REINGESTION_SEMAPHORE.release(); - return Response.status(Response.Status.CONFLICT) - .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") - .build(); - } + if (RUNNING_JOBS.size() >= MAX_PARALLEL_REINGESTIONS) { + return Response.status(Response.Status.TOO_MANY_REQUESTS) + .entity("Reingestion jobs parallel limit " + MAX_PARALLEL_REINGESTIONS + " reached.").build(); + } - InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); - if (instanceDataManager == null) { - throw new WebApplicationException(new RuntimeException("Invalid server initialization"), - Response.Status.INTERNAL_SERVER_ERROR); - } + InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); + if (instanceDataManager == null) { + throw new WebApplicationException("Invalid server initialization", Response.Status.INTERNAL_SERVER_ERROR); + } - TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); - if (tableDataManager == null) { - throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } + TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); + if (tableDataManager == null) { + throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } - IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); - LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); + IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); + LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); - // Get TableConfig and Schema - TableConfig tableConfig = indexLoadingConfig.getTableConfig(); - if (tableConfig == null) { - throw new WebApplicationException("Table config not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } + // Get TableConfig, Schema, ZK metadata + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + if (tableConfig == null) { + throw new WebApplicationException("Table config not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + Schema schema = indexLoadingConfig.getSchema(); + if (schema == null) { + throw new WebApplicationException("Schema not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } - Schema schema = indexLoadingConfig.getSchema(); - if (schema == null) { - throw new WebApplicationException("Schema not found for table: " + tableNameWithType, - Response.Status.NOT_FOUND); - } + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (segmentZKMetadata == null) { + throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, + Response.Status.NOT_FOUND); + } - // Fetch SegmentZKMetadata - SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); - if (segmentZKMetadata == null) { - throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, - Response.Status.NOT_FOUND); - } + // Grab start/end offsets + String startOffsetStr = segmentZKMetadata.getStartOffset(); + String endOffsetStr = segmentZKMetadata.getEndOffset(); + if (startOffsetStr == null || endOffsetStr == null) { + throw new WebApplicationException("Null start/end offset for segment: " + segmentName, + Response.Status.INTERNAL_SERVER_ERROR); + } - // Get startOffset, endOffset, partitionGroupId - String startOffsetStr = segmentZKMetadata.getStartOffset(); - String endOffsetStr = segmentZKMetadata.getEndOffset(); + // Check if this segment is already being re-ingested + AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); + if (!isIngesting.compareAndSet(false, true)) { + return Response.status(Response.Status.CONFLICT) + .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") + .build(); + } - if (startOffsetStr == null || endOffsetStr == null) { - return Response.serverError().entity("Start offset or end offset is null for segment: " + segmentName).build(); - } + // Generate a jobId for tracking + String jobId = UUID.randomUUID().toString(); + ReIngestionJob job = new ReIngestionJob(jobId, tableNameWithType, segmentName); + RUNNING_JOBS.put(jobId, job); - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); - int partitionGroupId = llcSegmentName.getPartitionGroupId(); + // Send immediate success response with jobId + ReIngestionResponse immediateResponse = new ReIngestionResponse( + "Re-ingestion job submitted successfully with jobId: " + jobId); + Response response = Response.ok(immediateResponse).build(); - Map streamConfigMap; + // Kick off the actual work asynchronously + REINGESTION_EXECUTOR.submit(() -> { try { - streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); - } catch (Exception e) { - return Response.serverError().entity("Failed to get stream config for table: " + tableNameWithType).build(); - } + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + int partitionGroupId = llcSegmentName.getPartitionGroupId(); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - // Set up directories - File resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); - File resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); + SimpleRealtimeSegmentDataManager manager = new SimpleRealtimeSegmentDataManager( + segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, tableConfig, schema, + indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, _serverInstance.getServerMetrics()); - if (!resourceTmpDir.exists()) { - resourceTmpDir.mkdirs(); - } - if (!resourceDataDir.exists()) { - resourceDataDir.mkdirs(); + doReIngestSegment(manager, segmentName, tableNameWithType, indexLoadingConfig, tableDataManager); + } catch (Exception e) { + LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); + } finally { + isIngesting.set(false); + RUNNING_JOBS.remove(jobId); } + }); - LOGGER.info("Starting SimpleRealtimeSegmentDataManager..."); - // Instantiate SimpleRealtimeSegmentDataManager - SimpleRealtimeSegmentDataManager manager = - new SimpleRealtimeSegmentDataManager(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, - tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, resourceTmpDir, - resourceDataDir, _serverInstance.getServerMetrics()); + // Return immediately with the jobId + return response; + } - try { + /** + * The actual re-ingestion logic, moved into a separate method for clarity. + * This is essentially the old synchronous logic you had in reIngestSegment. + */ + private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String segmentName, String tableNameWithType, + IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) + throws Exception { + try { + manager.startConsumption(); + waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300_000, 0); + manager.stopConsumption(); - manager.startConsumption(); + if (!manager.isSuccess()) { + throw new Exception("Consumer failed: " + manager.getConsumptionException()); + } - waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300000, 0); + LOGGER.info("Starting build for segment {}", segmentName); + SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + manager.buildSegmentInternal(); - manager.stopConsumption(); + File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + if (segmentTarFile == null) { + throw new Exception("Failed to build segment: " + segmentName); + } - // After ingestion is complete, get the segment - if (!manager.isSuccess()) { - throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); - } + ServerSegmentCompletionProtocolHandler protocolHandler = + new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); - LOGGER.info("Starting build for segment {}", segmentName); - SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = - manager.buildSegmentInternal(); + AuthProvider authProvider = protocolHandler.getAuthProvider(); + List
headers = AuthProviderUtils.toRequestHeaders(authProvider); - // Get the segment directory - File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); - if (segmentTarFile == null) { - throw new Exception("Failed to build segment: " + segmentName); - } + String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); + String segmentStoreUri = indexLoadingConfig.getSegmentStoreURI(); + String destUriStr = StringUtil.join(File.separator, segmentStoreUri, rawTableName, + SegmentCompletionUtils.generateTmpSegmentFileName(segmentName)); - ServerSegmentCompletionProtocolHandler protocolHandler = - new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); - - AuthProvider authProvider = protocolHandler.getAuthProvider(); - List
headers = AuthProviderUtils.toRequestHeaders(authProvider); - - String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); - - String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); - String segmentStoreUri = indexLoadingConfig.getSegmentStoreURI(); - String destUriStr = StringUtil.join(File.separator, segmentStoreUri, rawTableName, - SegmentCompletionUtils.generateTmpSegmentFileName(segmentName)); - try (PinotFS pinotFS = PinotFSFactory.create(new URI(segmentStoreUri).getScheme())) { - // copy segment to deep store - URI destUri = new URI(destUriStr); - if (pinotFS.exists(destUri)) { - pinotFS.delete(destUri, true); - } - pinotFS.copyFromLocalFile(segmentTarFile, destUri); - } catch (Exception e) { - throw new IOException("Failed to copy segment to deep store: " + destUriStr, e); + try (PinotFS pinotFS = PinotFSFactory.create(new URI(segmentStoreUri).getScheme())) { + URI destUri = new URI(destUriStr); + if (pinotFS.exists(destUri)) { + pinotFS.delete(destUri, true); } + pinotFS.copyFromLocalFile(segmentTarFile, destUri); + } - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); - pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); - - LOGGER.info("Segment metadata pushed, waiting for segment to be uploaded"); - // wait for segment metadata to have status as UPLOADED - waitForCondition((Void) -> { - SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); - if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { - return false; - } - - SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName); - return segmentDataManager instanceof ImmutableSegmentDataManager; - }, 5000, 300000, 0); - - // trigger segment reset call on API - LOGGER.info("Triggering segment reset for uploaded segment {}", segmentName); - HttpClient httpClient = HttpClient.getInstance(); - Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); - resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); + pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); - LOGGER.info("Re-ingested Segment {} uploaded successfully", segmentName); - } catch (Exception e) { - return Response.serverError().entity("Error during re-ingestion: " + e.getMessage()).build(); - } finally { - // Clean up - manager.offload(); - manager.destroy(); + // Wait for segment to be uploaded + waitForCondition((Void) -> { + SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { + return false; + } + SegmentDataManager segDataManager = tableDataManager.acquireSegment(segmentName); + return segDataManager instanceof ImmutableSegmentDataManager; + }, 5000, 300_000, 0); - // Delete temporary directories - FileUtils.deleteQuietly(resourceTmpDir); - FileUtils.deleteQuietly(resourceDataDir); + // Trigger segment reset + HttpClient httpClient = HttpClient.getInstance(); + Map headersMap = headers.stream() + .collect(Collectors.toMap(Header::getName, Header::getValue)); + resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); - isIngesting.set(false); - } - // Return success response - return Response.ok().entity(new ReIngestionResponse("Segment re-ingested and uploaded successfully")).build(); - } catch (Exception e) { - LOGGER.error("Error during re-ingestion", e); - throw new WebApplicationException(e, Response.Status.INTERNAL_SERVER_ERROR); + LOGGER.info("Re-ingested segment {} uploaded successfully", segmentName); } finally { - REINGESTION_SEMAPHORE.release(); + manager.offload(); + manager.destroy(); } } @@ -355,7 +386,6 @@ public void resetSegment(HttpClient httpClient, String controllerVipUrl, String String targetInstance, Map headers) throws IOException { try { - //TODO: send correct headers HttpClient.wrapAndThrowHttpException(httpClient.sendJsonPostRequest( new URI(getURLForSegmentReset(controllerVipUrl, tableNameWithType, segmentName, targetInstance)), null, headers)); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java index 23b4b49e111c..34e17e276e2f 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java @@ -119,7 +119,7 @@ public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, - File resourceTmpDir, File resourceDataDir, ServerMetrics serverMetrics) + ServerMetrics serverMetrics) throws Exception { _segmentName = segmentName; @@ -130,8 +130,8 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith _schema = schema; _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI(); _streamConfig = streamConfig; - _resourceTmpDir = resourceTmpDir; - _resourceDataDir = resourceDataDir; + _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); + _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis());; _serverMetrics = serverMetrics; _logger = LoggerFactory.getLogger(SimpleRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); @@ -141,6 +141,10 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith String clientId = getClientId(); + // Temp dirs + _resourceTmpDir.mkdirs(); + _resourceDataDir.mkdirs(); + _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig); _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId); _segmentNameStr = _segmentZKMetadata.getSegmentName(); @@ -168,7 +172,7 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith // much more efficient in allocating buffers. It also works with empty file String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != null ? indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + File.separator + _tableNameWithType - : resourceTmpDir.getAbsolutePath(); + : _resourceTmpDir.getAbsolutePath(); File statsHistoryFile = new File(tableDataDir, "segment-stats.ser"); RealtimeSegmentStatsHistory statsHistory = RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile); @@ -327,6 +331,8 @@ public String getSegmentName() { @Override protected void doDestroy() { _realtimeSegment.destroy(); + FileUtils.deleteQuietly(_resourceTmpDir); + FileUtils.deleteQuietly(_resourceDataDir); } @Override From 609942dd67e3d77cafac13e93f12f3f1fbebaa8d Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 27 Jan 2025 21:33:38 +0530 Subject: [PATCH 38/65] Refactor: fix doc comments --- .../core/realtime/PinotLLCRealtimeSegmentManager.java | 8 +++----- .../pinot/server/api/resources/ReIngestionResource.java | 6 ++++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index c51dda97fa37..62b2876f9e81 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2199,8 +2199,8 @@ URI createSegmentPath(String rawTableName, String segmentName) { } /** - * Re-ingests segments that are in DONE status with a missing download URL, but also - * have no peer copy on any server. This method will call the server reIngestSegment API + * Re-ingests segments that are in ERROR state in EV but ONLINE in IS with no peer copy on any server. This method + * will call the server reIngestSegment API * on one of the alive servers that are supposed to host that segment according to IdealState. * * API signature: @@ -2208,9 +2208,7 @@ URI createSegmentPath(String rawTableName, String segmentName) { * Request body (JSON): * { * "tableNameWithType": [tableName], - * "segmentName": [segmentName], - * "uploadURI": [leadControllerUrl], - * "uploadSegment": true + * "segmentName": [segmentName] * } * * @param tableNameWithType The table name with type, e.g. "myTable_REALTIME" diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 9b8b6600f466..2d9eb10f5844 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -128,6 +128,8 @@ public class ReIngestionResource { // Keep track of jobs by jobId => job info private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); + public static final long CONSUMPTION_END_TIMEOUT_MS = 300000L; + public static final long UPLOAD_END_TIMEOUT_MS = 300000L; @Inject private ServerInstance _serverInstance; @@ -290,7 +292,7 @@ private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String throws Exception { try { manager.startConsumption(); - waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300_000, 0); + waitForCondition((Void) -> manager.isDoneConsuming(), 1000, CONSUMPTION_END_TIMEOUT_MS, 0); manager.stopConsumption(); if (!manager.isSuccess()) { @@ -338,7 +340,7 @@ private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String } SegmentDataManager segDataManager = tableDataManager.acquireSegment(segmentName); return segDataManager instanceof ImmutableSegmentDataManager; - }, 5000, 300_000, 0); + }, 5000, UPLOAD_END_TIMEOUT_MS, 0); // Trigger segment reset HttpClient httpClient = HttpClient.getInstance(); From f939714a106530940a8409dc0e0cd3704743e11b Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" Date: Mon, 27 Jan 2025 16:47:51 -0800 Subject: [PATCH 39/65] Make SegmentZKMetadata JSON serializable --- .../metadata/segment/SegmentZKMetadata.java | 25 +++ .../segment/SimpleSegmentMetadata.java | 174 ------------------ .../utils/FileUploadDownloadClient.java | 9 +- .../PinotLLCRealtimeSegmentManager.java | 10 +- .../PinotLLCRealtimeSegmentManagerTest.java | 4 +- .../server/api/resources/TablesResource.java | 3 +- 6 files changed, 34 insertions(+), 191 deletions(-) delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java index 69cfe0a63a28..72df7fd9b479 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java @@ -18,6 +18,10 @@ */ package org.apache.pinot.common.metadata.segment; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.io.IOException; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.TimeUnit; @@ -33,6 +37,8 @@ public class SegmentZKMetadata implements ZKMetadata { private static final Logger LOGGER = LoggerFactory.getLogger(SegmentZKMetadata.class); + private static final String SEGMENT_NAME_KEY = "segmentName"; + private static final String SIMPLE_FIELDS_KEY = "simpleFields"; private static final String NULL = "null"; private final ZNRecord _znRecord; @@ -370,6 +376,25 @@ public Map toMap() { return metadataMap; } + public String toJsonString() { + ObjectNode objectNode = JsonUtils.newObjectNode(); + objectNode.put(SEGMENT_NAME_KEY, getSegmentName()); + objectNode.set(SIMPLE_FIELDS_KEY, JsonUtils.objectToJsonNode(_simpleFields)); + return objectNode.toString(); + } + + public static SegmentZKMetadata fromJsonString(String jsonString) + throws IOException { + JsonNode jsonNode = JsonUtils.stringToJsonNode(jsonString); + String segmentName = jsonNode.get(SEGMENT_NAME_KEY).asText(); + JsonNode simpleFieldsJsonNode = jsonNode.get(SIMPLE_FIELDS_KEY); + Map simpleFields = JsonUtils.jsonNodeToObject(simpleFieldsJsonNode, new TypeReference<>() { + }); + ZNRecord znRecord = new ZNRecord(segmentName); + znRecord.setSimpleFields(simpleFields); + return new SegmentZKMetadata(znRecord); + } + @Override public ZNRecord toZNRecord() { // Convert to TreeMap to keep the keys sorted. The de-serialized ZNRecord has simple fields stored as LinkedHashMap. diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java deleted file mode 100644 index 5d009bec96a4..000000000000 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SimpleSegmentMetadata.java +++ /dev/null @@ -1,174 +0,0 @@ -/** - * 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.common.metadata.segment; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonGetter; -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.TimeUnit; -import org.apache.pinot.spi.utils.CommonConstants; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * A simplified version of SegmentZKMetadata designed for easy serialization and deserialization. - * This class maintains only the essential fields from SegmentZKMetadata while providing JSON - * serialization support via Jackson annotations. - * - * Use cases: - * 1. Serializing segment metadata for API responses - * 2. Transferring segment metadata between services - * 3. Storing segment metadata in a format that's easy to deserialize - * - * Example usage: - * SimpleSegmentMetadata metadata = SimpleSegmentMetadata.fromZKMetadata(zkMetadata); - * String json = JsonUtils.objectToString(metadata); - * SimpleSegmentMetadata deserialized = objectMapper.readValue(json, SimpleSegmentMetadata.class); - */ -@JsonIgnoreProperties(ignoreUnknown = true) -public class SimpleSegmentMetadata { - private static final Logger LOGGER = LoggerFactory.getLogger(SimpleSegmentMetadata.class); - private final String _segmentName; - private final Map _simpleFields; - - @JsonCreator - public SimpleSegmentMetadata( - @JsonProperty("segmentName") String segmentName, - @JsonProperty("simpleFields") Map simpleFields) { - _segmentName = segmentName; - _simpleFields = new HashMap<>(simpleFields); - } - - @JsonGetter - public String getSegmentName() { - return _segmentName; - } - - @JsonGetter - public Map getSimpleFields() { - return Collections.unmodifiableMap(_simpleFields); - } - - public long getStartTimeMs() { - long startTimeMs = -1; - String startTimeString = _simpleFields.get(CommonConstants.Segment.START_TIME); - if (startTimeString != null) { - long startTime = Long.parseLong(startTimeString); - if (startTime > 0) { - // NOTE: Need to check whether the start time is positive because some old segment ZK metadata contains negative - // start time and null time unit - startTimeMs = TimeUnit.valueOf(_simpleFields.get(CommonConstants.Segment.TIME_UNIT)).toMillis(startTime); - } - } - return startTimeMs; - } - - public long getEndTimeMs() { - long endTimeMs = -1; - String endTimeString = _simpleFields.get(CommonConstants.Segment.END_TIME); - if (endTimeString != null) { - long endTime = Long.parseLong(endTimeString); - // NOTE: Need to check whether the end time is positive because some old segment ZK metadata contains negative - // end time and null time unit - if (endTime > 0) { - endTimeMs = TimeUnit.valueOf(_simpleFields.get(CommonConstants.Segment.TIME_UNIT)).toMillis(endTime); - } - } - return endTimeMs; - } - - public String getIndexVersion() { - return _simpleFields.get(CommonConstants.Segment.INDEX_VERSION); - } - - public long getTotalDocs() { - String value = _simpleFields.get(CommonConstants.Segment.TOTAL_DOCS); - return value != null ? Long.parseLong(value) : -1; - } - - public SegmentPartitionMetadata getPartitionMetadata() { - String partitionMetadataJson = _simpleFields.get(CommonConstants.Segment.PARTITION_METADATA); - if (partitionMetadataJson != null) { - try { - return SegmentPartitionMetadata.fromJsonString(partitionMetadataJson); - } catch (Exception e) { - LOGGER.error("Caught exception while reading partition metadata for segment: {}", getSegmentName(), e); - } - } - return null; - } - - public long getSizeInBytes() { - String value = _simpleFields.get(CommonConstants.Segment.SIZE_IN_BYTES); - return value != null ? Long.parseLong(value) : -1; - } - - public long getCrc() { - String value = _simpleFields.get(CommonConstants.Segment.CRC); - return value != null ? Long.parseLong(value) : -1; - } - - public String getDownloadUrl() { - String downloadUrl = _simpleFields.get(CommonConstants.Segment.DOWNLOAD_URL); - // Handle legacy download url keys - if (downloadUrl == null) { - downloadUrl = _simpleFields.get(CommonConstants.Segment.Offline.DOWNLOAD_URL); - if (downloadUrl == null) { - downloadUrl = _simpleFields.get(CommonConstants.Segment.Realtime.DOWNLOAD_URL); - } - } - return downloadUrl; - } - - /** - * Creates a SimpleSegmentMetadata instance from a SegmentZKMetadata object. - * This method copies all simple fields from the ZK metadata while maintaining - * the immutability guarantees of this class. - */ - public static SimpleSegmentMetadata fromZKMetadata(SegmentZKMetadata zkMetadata) { - return new SimpleSegmentMetadata( - zkMetadata.getSegmentName(), - zkMetadata.toMap() - ); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SimpleSegmentMetadata that = (SimpleSegmentMetadata) o; - return Objects.equals(_segmentName, that._segmentName) - && Objects.equals(_simpleFields, that._simpleFields); - } - - @Override - public int hashCode() { - return Objects.hash(_segmentName, _simpleFields); - } -} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 02bd02002285..dc68925617f5 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -56,7 +56,6 @@ import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.restlet.resources.EndReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.StartReplaceSegmentsRequest; import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; @@ -1002,16 +1001,14 @@ public TableLLCSegmentUploadResponse uploadLLCToSegmentStore(String uri) * @throws IOException * @throws HttpErrorStatusException */ - public SimpleSegmentMetadata uploadLLCToSegmentStoreWithZKMetadata(String uri) + public SegmentZKMetadata uploadLLCToSegmentStoreWithZKMetadata(String uri) throws URISyntaxException, IOException, HttpErrorStatusException { ClassicRequestBuilder requestBuilder = ClassicRequestBuilder.post(new URI(uri)).setVersion(HttpVersion.HTTP_1_1); // sendRequest checks the response status code SimpleHttpResponse response = HttpClient.wrapAndThrowHttpException( _httpClient.sendRequest(requestBuilder.build(), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS)); - SimpleSegmentMetadata segmentZKMetadata = - JsonUtils.stringToObject(response.getResponse(), SimpleSegmentMetadata.class); - if (segmentZKMetadata.getDownloadUrl() == null - || segmentZKMetadata.getDownloadUrl().isEmpty()) { + SegmentZKMetadata segmentZKMetadata = SegmentZKMetadata.fromJsonString(response.getResponse()); + if (StringUtils.isEmpty(segmentZKMetadata.getDownloadUrl())) { throw new HttpErrorStatusException( String.format("Returned segment download url is empty after requesting servers to upload by the path: %s", uri), response.getStatusCode()); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 62b2876f9e81..433dec23b29e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -64,7 +64,6 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.metrics.ControllerGauge; import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; @@ -1782,7 +1781,7 @@ private void uploadToDeepStoreWithFallback(URI uri, String segmentName, String r String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegmentToDeepStore"); LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, serverUploadRequestUrl); - SimpleSegmentMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( + SegmentZKMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( serverUploadRequestUrl); handleMetadataUpload(segmentName, rawTableName, segmentZKMetadata, uploadedMetadata, pinotFS); }, @@ -1827,8 +1826,8 @@ private String getUploadUrl(URI uri, String endpoint) { uri.toString(), endpoint, _deepstoreUploadRetryTimeoutMs); } - private void handleMetadataUpload(String segmentName, String rawTableName, - SegmentZKMetadata currentMetadata, SimpleSegmentMetadata uploadedMetadata, + private void handleMetadataUpload(String segmentName, String rawTableName, SegmentZKMetadata currentMetadata, + SegmentZKMetadata uploadedMetadata, PinotFS pinotFS) throws Exception { @@ -1881,8 +1880,7 @@ private void handleBasicUpload(String segmentName, String rawTableName, * @param segmentZKMetadata Current segment metadata stored in ZooKeeper that needs to be updated * @param uploadedSegmentZKMetadata New metadata from the successfully uploaded segment */ - private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, - SimpleSegmentMetadata uploadedSegmentZKMetadata) { + private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata uploadedSegmentZKMetadata) { if (segmentZKMetadata.getStatus() == Status.COMMITTING) { LOGGER.info("Updating additional metadata in ZK for segment {} as pauseless is enabled", segmentZKMetadata.getSegmentName()); diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index aed43c0401c4..18710107dd96 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -51,7 +51,6 @@ import org.apache.pinot.common.assignment.InstancePartitions; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.restlet.resources.TableLLCSegmentUploadResponse; import org.apache.pinot.common.utils.FileUploadDownloadClient; @@ -1249,8 +1248,7 @@ public void testUploadLLCSegmentToDeepStore() segmentZKMetadataCopy.setDownloadUrl(tempSegmentFileLocation.getPath()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( - serverUploadRequestUrl0)).thenReturn( - SimpleSegmentMetadata.fromZKMetadata(segmentZKMetadataCopy)); + serverUploadRequestUrl0)).thenReturn(segmentZKMetadataCopy); // Change 2nd segment status to be DONE, but with default peer download url. // Verify later the download url isn't fixed after upload failure. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 92c522e8a50c..f7a7048ecba4 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -65,7 +65,6 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metadata.segment.SimpleSegmentMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; @@ -1065,7 +1064,7 @@ public String uploadLLCSegmentToDeepStore( segmentName), Response.Status.INTERNAL_SERVER_ERROR); } segmentZKMetadata.setDownloadUrl(segmentDownloadUrl.toString()); - return JsonUtils.objectToPrettyString(SimpleSegmentMetadata.fromZKMetadata(segmentZKMetadata)); + return segmentZKMetadata.toJsonString(); } finally { FileUtils.deleteQuietly(segmentTarFile); tableDataManager.releaseSegment(segmentDataManager); From 155c49f6fcbeedb0c06453749a1693cbdf602706 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" Date: Mon, 27 Jan 2025 17:10:31 -0800 Subject: [PATCH 40/65] Minor API name change --- .../controller/helix/core/PinotHelixResourceManager.java | 6 +++--- .../tests/BasePauselessRealtimeIngestionTest.java | 6 ++---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 095db4dc4ec5..95dc452703f2 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -406,11 +406,11 @@ public ZkHelixPropertyStore getPropertyStore() { } /** - * Get the Pinot llc realtime segment manager + * Get the realtime segment manager * - * @return Pinot llc realtime segment manager + * @return Realtime segment manager */ - public PinotLLCRealtimeSegmentManager getPinotLLCRealtimeSegmentManager() { + public PinotLLCRealtimeSegmentManager getRealtimeSegmentManager() { return _pinotLLCRealtimeSegmentManager; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index 38ad2f7e47cc..95e8e95f6c06 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -154,15 +154,13 @@ private void setupPauselessTable() } protected void injectFailure() { - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .enableTestFault(getFailurePoint()); + _helixResourceManager.getRealtimeSegmentManager().enableTestFault(getFailurePoint()); _failureEnabled = true; } protected void disableFailure() { _failureEnabled = false; - _helixResourceManager.getPinotLLCRealtimeSegmentManager() - .disableTestFault(getFailurePoint()); + _helixResourceManager.getRealtimeSegmentManager().disableTestFault(getFailurePoint()); } @AfterClass From 080ec5595fd2b10010794e14c8789a1a7cfce575 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 28 Jan 2025 09:43:41 +0530 Subject: [PATCH 41/65] Refactor PinotLLC class to add ability to inject failures --- .../controller/BaseControllerStarter.java | 7 +- .../PinotLLCRealtimeSegmentManager.java | 40 +++++------ .../BasePauselessRealtimeIngestionTest.java | 19 +++++- .../FailureInjectingControllerStarter.java | 30 +++++++++ ...jectingPinotLLCRealtimeSegmentManager.java | 66 +++++++++++++++++++ 5 files changed, 136 insertions(+), 26 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingControllerStarter.java create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java index 171e8506387a..a0ce503d41a1 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java @@ -487,8 +487,7 @@ private void setUpPinotController() { // Helix resource manager must be started in order to create PinotLLCRealtimeSegmentManager LOGGER.info("Starting realtime segment manager"); - _pinotLLCRealtimeSegmentManager = - new PinotLLCRealtimeSegmentManager(_helixResourceManager, _config, _controllerMetrics); + _pinotLLCRealtimeSegmentManager = createPinotLLCRealtimeSegmentManager(); // TODO: Need to put this inside HelixResourceManager when HelixControllerLeadershipManager is removed. _helixResourceManager.registerPinotLLCRealtimeSegmentManager(_pinotLLCRealtimeSegmentManager); @@ -623,6 +622,10 @@ protected void configure() { _serviceStatusCallbackList.add(generateServiceStatusCallback(_helixParticipantManager)); } + protected PinotLLCRealtimeSegmentManager createPinotLLCRealtimeSegmentManager() { + return new PinotLLCRealtimeSegmentManager(_helixResourceManager, _config, _controllerMetrics); + } + /** * This method is used to fix table/schema names. * TODO: in the next release, maybe 2.0.0, we can remove this method. Meanwhile we can delete the orphan schemas diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 433dec23b29e..bab051c7e182 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -88,7 +88,6 @@ import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpdater; import org.apache.pinot.controller.helix.core.retention.strategy.RetentionStrategy; import org.apache.pinot.controller.helix.core.retention.strategy.TimeRetentionStrategy; -import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; import org.apache.pinot.controller.validation.RealtimeSegmentValidationManager; import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.core.util.PeerServerSegmentFinder; @@ -194,8 +193,6 @@ public class PinotLLCRealtimeSegmentManager { private final AtomicInteger _numCompletingSegments = new AtomicInteger(0); private final ExecutorService _deepStoreUploadExecutor; private final Set _deepStoreUploadExecutorPendingSegments; - @VisibleForTesting - private final Map _failureConfig; private volatile boolean _isStopping = false; @@ -220,7 +217,6 @@ public PinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceMan controllerConf.getDeepStoreRetryUploadParallelism()) : null; _deepStoreUploadExecutorPendingSegments = _isDeepStoreLLCSegmentUploadRetryEnabled ? ConcurrentHashMap.newKeySet() : null; - _failureConfig = new HashMap<>(); } public boolean isDeepStoreLLCSegmentUploadRetryEnabled() { @@ -555,10 +551,7 @@ private void commitSegmentMetadataInternal(String realtimeTableName, SegmentZKMetadata committingSegmentZKMetadata = updateCommittingSegmentMetadata(realtimeTableName, committingSegmentDescriptor, isStartMetadata); - // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the - // segment that encounter failure at this stage of commit protocol. - FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION, - _failureConfig); + preProcessNewSegmentZKMetadata(); // Step-2: Create new segment metadata if needed LOGGER.info("Creating new segment metadata with status IN_PROGRESS: {}", committingSegmentName); @@ -567,9 +560,7 @@ private void commitSegmentMetadataInternal(String realtimeTableName, createNewSegmentMetadata(tableConfig, idealState, committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions); - // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the - // segment that encounter failure at this stage of commit protocol. - FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE, _failureConfig); + preProcessCommitIdealStateUpdate(); // Step-3: Update IdealState LOGGER.info("Updating Idealstate for previous: {} and new segment: {}", committingSegmentName, @@ -609,6 +600,21 @@ private void commitSegmentMetadataInternal(String realtimeTableName, } } + @VisibleForTesting + protected void preProcessNewSegmentZKMetadata() { + // No-op + } + + @VisibleForTesting + protected void preProcessCommitIdealStateUpdate() { + // No-op + } + + @VisibleForTesting + protected void preProcessCommitSegmentEndMetadata() { + // No-op + } + // Step 1: Update committing segment metadata private SegmentZKMetadata updateCommittingSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor, boolean isStartMetadata) { @@ -704,7 +710,7 @@ public void commitSegmentEndMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the // segment that encounter failure at this stage of commit protocol. - FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA, _failureConfig); + preProcessCommitSegmentEndMetadata(); Preconditions.checkState(!_isStopping, "Segment manager is stopping"); try { _numCompletingSegments.addAndGet(1); @@ -2327,16 +2333,6 @@ private String findAliveServerToReIngest(Set candidateServers) { return null; } - @VisibleForTesting - public void enableTestFault(String faultType) { - _failureConfig.put(faultType, "true"); - } - - @VisibleForTesting - public void disableTestFault(String faultType) { - _failureConfig.remove(faultType); - } - public Set getSegmentsYetToBeCommitted(String tableNameWithType, Set segmentsToCheck) { Set segmentsYetToBeCommitted = new HashSet<>(); for (String segmentName: segmentsToCheck) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index 95e8e95f6c06..f84498332d16 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -27,8 +27,12 @@ import org.apache.commons.io.FileUtils; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.PauselessConsumptionUtils; +import org.apache.pinot.controller.BaseControllerStarter; import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingControllerStarter; +import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingPinotLLCRealtimeSegmentManager; import org.apache.pinot.integration.tests.realtime.utils.PauselessRealtimeTestUtils; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -66,6 +70,11 @@ public abstract class BasePauselessRealtimeIngestionTest extends BaseClusterInte protected abstract long getCountStarResultWithFailure(); + @Override + public BaseControllerStarter createControllerStarter() { + return new FailureInjectingControllerStarter(); + } + @Override protected void overrideControllerConf(Map properties) { properties.put(ControllerConf.ControllerPeriodicTasksConf.PINOT_TASK_MANAGER_SCHEDULER_ENABLED, true); @@ -154,13 +163,19 @@ private void setupPauselessTable() } protected void injectFailure() { - _helixResourceManager.getRealtimeSegmentManager().enableTestFault(getFailurePoint()); + PinotLLCRealtimeSegmentManager realtimeSegmentManager = _helixResourceManager.getRealtimeSegmentManager(); + if (realtimeSegmentManager instanceof FailureInjectingPinotLLCRealtimeSegmentManager) { + ((FailureInjectingPinotLLCRealtimeSegmentManager) realtimeSegmentManager).enableTestFault(getFailurePoint()); + } _failureEnabled = true; } protected void disableFailure() { _failureEnabled = false; - _helixResourceManager.getRealtimeSegmentManager().disableTestFault(getFailurePoint()); + PinotLLCRealtimeSegmentManager realtimeSegmentManager = _helixResourceManager.getRealtimeSegmentManager(); + if (realtimeSegmentManager instanceof FailureInjectingPinotLLCRealtimeSegmentManager) { + ((FailureInjectingPinotLLCRealtimeSegmentManager) realtimeSegmentManager).disableTestFault(getFailurePoint()); + } } @AfterClass diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingControllerStarter.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingControllerStarter.java new file mode 100644 index 000000000000..468b956cb4e4 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingControllerStarter.java @@ -0,0 +1,30 @@ +/** + * 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.integration.tests.realtime.utils; + +import org.apache.pinot.controller.ControllerStarter; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; + + +public class FailureInjectingControllerStarter extends ControllerStarter { + @Override + protected PinotLLCRealtimeSegmentManager createPinotLLCRealtimeSegmentManager() { + return new FailureInjectingPinotLLCRealtimeSegmentManager(_helixResourceManager, getConfig(), _controllerMetrics); + } +} diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java new file mode 100644 index 000000000000..66573e34b543 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java @@ -0,0 +1,66 @@ +/** + * 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.integration.tests.realtime.utils; + +import com.google.common.annotations.VisibleForTesting; +import java.util.HashMap; +import java.util.Map; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; + + +public class FailureInjectingPinotLLCRealtimeSegmentManager extends PinotLLCRealtimeSegmentManager { + @VisibleForTesting + private final Map _failureConfig; + + public FailureInjectingPinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceManager, + ControllerConf controllerConf, ControllerMetrics controllerMetrics) { + super(helixResourceManager, controllerConf, controllerMetrics); + _failureConfig = new HashMap<>(); + } + + @VisibleForTesting + public void enableTestFault(String faultType) { + _failureConfig.put(faultType, "true"); + } + + @VisibleForTesting + public void disableTestFault(String faultType) { + _failureConfig.remove(faultType); + } + + @Override + protected void preProcessNewSegmentZKMetadata() { + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_NEW_SEGMENT_METADATA_CREATION, + _failureConfig); + } + + @Override + protected void preProcessCommitIdealStateUpdate() { + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_IDEAL_STATE_UPDATE, _failureConfig); + } + + @Override + protected void preProcessCommitSegmentEndMetadata() { + FailureInjectionUtils.injectFailure(FailureInjectionUtils.FAULT_BEFORE_COMMIT_END_METADATA, _failureConfig); + } +} From 7e04fa304d7bee415a512af984cc06609f7d1168 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Tue, 28 Jan 2025 13:59:08 +0530 Subject: [PATCH 42/65] Minor improvements --- .../metadata/segment/SegmentZKMetadata.java | 10 +++++ .../PinotLLCRealtimeSegmentManager.java | 39 +++++++------------ .../server/api/resources/TablesResource.java | 14 ++++++- 3 files changed, 36 insertions(+), 27 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java index 72df7fd9b479..6373beb1cf39 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java @@ -395,6 +395,16 @@ public static SegmentZKMetadata fromJsonString(String jsonString) return new SegmentZKMetadata(znRecord); } + public void copySimpleFieldsFrom(SegmentZKMetadata segmentZKMetadata) { + ZNRecord znRecord = segmentZKMetadata.toZNRecord(); + _simpleFields.clear(); + _simpleFields.putAll(znRecord.getSimpleFields()); + + // Reset the cached values + _startTimeMsCached = false; + _endTimeMsCached = false; + } + @Override public ZNRecord toZNRecord() { // Convert to TreeMap to keep the keys sorted. The de-serialized ZNRecord has simple fields stored as LinkedHashMap. diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index bab051c7e182..9629199a6573 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -1398,17 +1398,19 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, List entry : latestSegmentZKMetadataMap.entrySet()) { int partitionId = entry.getKey(); SegmentZKMetadata latestSegmentZKMetadata = entry.getValue(); String latestSegmentName = latestSegmentZKMetadata.getSegmentName(); LLCSegmentName latestLLCSegmentName = new LLCSegmentName(latestSegmentName); - // This is the expected segment status after completion of first of the 3 steps of the segment commit protocol - // The status in step one is updated to - // 1. DONE for normal consumption - // 2. COMMITTING for pauseless consumption - Status statusPostSegmentMetadataUpdate = - PauselessConsumptionUtils.isPauselessEnabled(tableConfig) ? Status.COMMITTING : Status.DONE; Map instanceStateMap = instanceStatesMap.get(latestSegmentName); if (instanceStateMap != null) { @@ -1837,15 +1839,15 @@ private void handleMetadataUpload(String segmentName, String rawTableName, Segme PinotFS pinotFS) throws Exception { - String downloadUrl = moveSegmentFile(rawTableName, segmentName, uploadedMetadata.getDownloadUrl(), pinotFS); - LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); - currentMetadata.setDownloadUrl(downloadUrl); - if (uploadedMetadata.getCrc() != currentMetadata.getCrc()) { LOGGER.info("Updating segment {} crc in ZK to be {} from previous {}", segmentName, uploadedMetadata.getCrc(), currentMetadata.getCrc()); updateSegmentMetadata(currentMetadata, uploadedMetadata); } + + String downloadUrl = moveSegmentFile(rawTableName, segmentName, uploadedMetadata.getDownloadUrl(), pinotFS); + LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); + currentMetadata.setDownloadUrl(downloadUrl); } private void handleLLCUpload(String segmentName, String rawTableName, @@ -1890,21 +1892,8 @@ private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, SegmentZ if (segmentZKMetadata.getStatus() == Status.COMMITTING) { LOGGER.info("Updating additional metadata in ZK for segment {} as pauseless is enabled", segmentZKMetadata.getSegmentName()); - segmentZKMetadata.setStartTime(uploadedSegmentZKMetadata.getStartTimeMs()); - segmentZKMetadata.setEndTime(uploadedSegmentZKMetadata.getEndTimeMs()); - segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); - - if (uploadedSegmentZKMetadata.getIndexVersion() != null) { - segmentZKMetadata.setIndexVersion(uploadedSegmentZKMetadata.getIndexVersion()); - } - segmentZKMetadata.setTotalDocs(uploadedSegmentZKMetadata.getTotalDocs()); - segmentZKMetadata.setPartitionMetadata(uploadedSegmentZKMetadata.getPartitionMetadata()); - - // set the size that can be utilized for size based segment thresholds. - segmentZKMetadata.setSizeInBytes(uploadedSegmentZKMetadata.getSizeInBytes()); - - // The segment is now ONLINE on the server, so marking its status as DONE. - segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); + // Copy all the simple fields from the uploaded segment + segmentZKMetadata.copySimpleFieldsFrom(uploadedSegmentZKMetadata); } segmentZKMetadata.setCrc(uploadedSegmentZKMetadata.getCrc()); } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index f7a7048ecba4..1fa4f6de4c67 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -1032,7 +1032,10 @@ public String uploadLLCSegmentToDeepStore( ImmutableSegmentDataManager immutableSegmentDataManager = (ImmutableSegmentDataManager) segmentDataManager; SegmentMetadataImpl segmentMetadata = (SegmentMetadataImpl) immutableSegmentDataManager.getSegment().getSegmentMetadata(); - SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(segmentMetadata); + SegmentZKMetadata existingSegmentZkMetadata = + ZKMetadataProvider.getSegmentZKMetadata(_serverInstance.getHelixManager().getHelixPropertyStore(), + realtimeTableNameWithType, segmentName); + SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(segmentMetadata, existingSegmentZkMetadata); segmentZKMetadata.setSizeInBytes(immutableSegmentDataManager.getSegment().getSegmentSizeBytes()); File segmentTarFile = null; try { @@ -1071,8 +1074,15 @@ public String uploadLLCSegmentToDeepStore( } } - private SegmentZKMetadata getSegmentZKMetadata(SegmentMetadataImpl segmentMetadata) { + private SegmentZKMetadata getSegmentZKMetadata(SegmentMetadataImpl segmentMetadata, + SegmentZKMetadata existingSegmentZKMetadata) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segmentMetadata.getName()); + + // fetch the creation time and num replicas from the existing segment zk metadata + // these fields are set when the segment ZK is created. + segmentZKMetadata.setCreationTime(existingSegmentZKMetadata.getCreationTime()); + segmentZKMetadata.setNumReplicas(existingSegmentZKMetadata.getNumReplicas()); + // set offsets for segment segmentZKMetadata.setStartOffset(segmentMetadata.getStartOffset()); segmentZKMetadata.setEndOffset(segmentMetadata.getEndOffset()); From 523913f9a2e9178cdffa43ad73c6ff3c81e0dd95 Mon Sep 17 00:00:00 2001 From: Aman Khanchandani Date: Tue, 28 Jan 2025 19:28:17 +0530 Subject: [PATCH 43/65] Moving ZkMetadaptaUtils to commons and reusing the code in the upload segment API's --- .../common/utils/helix}/ZKMetadataUtils.java | 2 +- .../controller/api/upload/ZKOperator.java | 2 +- .../helix/core/PinotHelixResourceManager.java | 2 +- .../utils/PauselessRealtimeTestUtils.java | 2 +- .../server/api/resources/TablesResource.java | 95 ++++--------------- 5 files changed, 22 insertions(+), 81 deletions(-) rename {pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util => pinot-common/src/main/java/org/apache/pinot/common/utils/helix}/ZKMetadataUtils.java (99%) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/ZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java similarity index 99% rename from pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/ZKMetadataUtils.java rename to pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java index d481b669a45d..f4f1e0d42b1e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/util/ZKMetadataUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.controller.helix.core.util; +package org.apache.pinot.common.utils.helix; import com.google.common.base.Preconditions; import java.util.HashMap; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index eeee40b20d49..ec39f136ad27 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -40,7 +40,7 @@ import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.exception.ControllerApplicationException; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; -import org.apache.pinot.controller.helix.core.util.ZKMetadataUtils; +import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.slf4j.Logger; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 95dc452703f2..d52e3fcfcd5e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -154,7 +154,7 @@ import org.apache.pinot.controller.helix.core.rebalance.TableRebalanceContext; import org.apache.pinot.controller.helix.core.rebalance.TableRebalancer; import org.apache.pinot.controller.helix.core.rebalance.ZkBasedTableRebalanceObserver; -import org.apache.pinot.controller.helix.core.util.ZKMetadataUtils; +import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.helix.starter.HelixConfig; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.config.DatabaseConfig; diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java index 4196b43dcfd7..fc7f5264c016 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/PauselessRealtimeTestUtils.java @@ -78,7 +78,7 @@ private static Map getPartitionSegmentNumberToMetadat private static void compareSegmentZkMetadata(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata segmentZKMetadata1) { - if (segmentZKMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.DONE) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.IN_PROGRESS) { return; } assertEquals(segmentZKMetadata.getStatus(), segmentZKMetadata1.getStatus()); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 1fa4f6de4c67..a3bfd661adef 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -33,16 +33,13 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import javax.annotation.Nullable; import javax.inject.Inject; import javax.inject.Named; import javax.ws.rs.DefaultValue; @@ -63,7 +60,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.ZKMetadataProvider; -import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; @@ -82,6 +78,7 @@ import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; @@ -94,13 +91,10 @@ import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; -import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.IndexService; import org.apache.pinot.segment.spi.index.IndexType; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; -import org.apache.pinot.segment.spi.partition.PartitionFunction; -import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.server.access.AccessControlFactory; import org.apache.pinot.server.api.AdminApiApplication; import org.apache.pinot.server.starter.ServerInstance; @@ -1029,14 +1023,7 @@ public String uploadLLCSegmentToDeepStore( String.format("Table %s segment %s does not exist on the disk", realtimeTableNameWithType, segmentName), Response.Status.NOT_FOUND); } - ImmutableSegmentDataManager immutableSegmentDataManager = (ImmutableSegmentDataManager) segmentDataManager; - SegmentMetadataImpl segmentMetadata = - (SegmentMetadataImpl) immutableSegmentDataManager.getSegment().getSegmentMetadata(); - SegmentZKMetadata existingSegmentZkMetadata = - ZKMetadataProvider.getSegmentZKMetadata(_serverInstance.getHelixManager().getHelixPropertyStore(), - realtimeTableNameWithType, segmentName); - SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(segmentMetadata, existingSegmentZkMetadata); - segmentZKMetadata.setSizeInBytes(immutableSegmentDataManager.getSegment().getSegmentSizeBytes()); + File segmentTarFile = null; try { // Create the tar.gz segment file in the server's segmentTarUploadDir folder with a unique file name. @@ -1066,7 +1053,22 @@ public String uploadLLCSegmentToDeepStore( String.format("Failed to upload table %s segment %s to segment store", realtimeTableNameWithType, segmentName), Response.Status.INTERNAL_SERVER_ERROR); } - segmentZKMetadata.setDownloadUrl(segmentDownloadUrl.toString()); + + ImmutableSegmentDataManager immutableSegmentDataManager = (ImmutableSegmentDataManager) segmentDataManager; + SegmentMetadataImpl segmentMetadata = + (SegmentMetadataImpl) immutableSegmentDataManager.getSegment().getSegmentMetadata(); + // fetch existing ZK Metadata + SegmentZKMetadata segmentZKMetadata = + ZKMetadataProvider.getSegmentZKMetadata(_serverInstance.getHelixManager().getHelixPropertyStore(), + realtimeTableNameWithType, segmentName); + + // Update the Segment ZK Metadata with the segment metadata present on the server + // along with download url and segment size + ZKMetadataUtils.refreshSegmentZKMetadata(realtimeTableNameWithType, segmentZKMetadata, segmentMetadata, + segmentDownloadUrl.toString(), null, immutableSegmentDataManager.getSegment().getSegmentSizeBytes()); + // mark the segment status as DONE + segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); + return segmentZKMetadata.toJsonString(); } finally { FileUtils.deleteQuietly(segmentTarFile); @@ -1074,67 +1076,6 @@ public String uploadLLCSegmentToDeepStore( } } - private SegmentZKMetadata getSegmentZKMetadata(SegmentMetadataImpl segmentMetadata, - SegmentZKMetadata existingSegmentZKMetadata) { - SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segmentMetadata.getName()); - - // fetch the creation time and num replicas from the existing segment zk metadata - // these fields are set when the segment ZK is created. - segmentZKMetadata.setCreationTime(existingSegmentZKMetadata.getCreationTime()); - segmentZKMetadata.setNumReplicas(existingSegmentZKMetadata.getNumReplicas()); - - // set offsets for segment - segmentZKMetadata.setStartOffset(segmentMetadata.getStartOffset()); - segmentZKMetadata.setEndOffset(segmentMetadata.getEndOffset()); - - // The segment is now ONLINE on the server, so marking its status as DONE. - segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); - segmentZKMetadata.setCrc(Long.parseLong(segmentMetadata.getCrc())); - - // set start and end time - // the time unit is always kept to MILLISECONDS when the controller commits the ZK Metadata - if (segmentMetadata.getTotalDocs() > 0) { - Preconditions.checkNotNull(segmentMetadata.getTimeInterval(), - "start/end time information is not correctly written to the segment for table: " - + segmentMetadata.getTableName()); - segmentZKMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis()); - segmentZKMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis()); - } else { - // Set current time as start/end time if total docs is 0 - long now = System.currentTimeMillis(); - segmentZKMetadata.setStartTime(now); - segmentZKMetadata.setEndTime(now); - } - segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); - - SegmentVersion segmentVersion = segmentMetadata.getVersion(); - if (segmentVersion != null) { - segmentZKMetadata.setIndexVersion(segmentVersion.name()); - } - segmentZKMetadata.setTotalDocs(segmentMetadata.getTotalDocs()); - - segmentZKMetadata.setPartitionMetadata(getPartitionMetadataFromSegmentMetadata(segmentMetadata)); - - return segmentZKMetadata; - } - - @Nullable - private SegmentPartitionMetadata getPartitionMetadataFromSegmentMetadata(SegmentMetadataImpl segmentMetadata) { - for (Map.Entry entry : segmentMetadata.getColumnMetadataMap().entrySet()) { - // NOTE: There is at most one partition column. - ColumnMetadata columnMetadata = entry.getValue(); - PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); - if (partitionFunction != null) { - ColumnPartitionMetadata columnPartitionMetadata = - new ColumnPartitionMetadata(partitionFunction.getName(), partitionFunction.getNumPartitions(), - columnMetadata.getPartitions(), columnMetadata.getPartitionFunction().getFunctionConfig()); - return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); - } - } - return null; - } - - @GET @Path("tables/{realtimeTableName}/consumingSegmentsInfo") @Produces(MediaType.APPLICATION_JSON) From 5689333612cc6f604160bc00d98e797a1e0b33ca Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 28 Jan 2025 22:36:45 +0530 Subject: [PATCH 44/65] Fix lint failures --- .../java/org/apache/pinot/controller/api/upload/ZKOperator.java | 2 +- .../pinot/controller/helix/core/PinotHelixResourceManager.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index ec39f136ad27..73049506ba1b 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -37,10 +37,10 @@ import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.FileUploadDownloadClient.FileUploadType; +import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.exception.ControllerApplicationException; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; -import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.slf4j.Logger; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index d52e3fcfcd5e..51724fa1a345 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -132,6 +132,7 @@ import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.common.utils.helix.PinotHelixPropertyStoreZnRecordProvider; +import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.exception.ControllerApplicationException; import org.apache.pinot.controller.api.exception.InvalidTableConfigException; @@ -154,7 +155,6 @@ import org.apache.pinot.controller.helix.core.rebalance.TableRebalanceContext; import org.apache.pinot.controller.helix.core.rebalance.TableRebalancer; import org.apache.pinot.controller.helix.core.rebalance.ZkBasedTableRebalanceObserver; -import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.helix.starter.HelixConfig; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.config.DatabaseConfig; From f42c6b89660a7de06ebb0c36f206226b3b626a88 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" Date: Tue, 28 Jan 2025 17:26:42 -0800 Subject: [PATCH 45/65] Misc fix and cleanup --- .../metadata/segment/SegmentZKMetadata.java | 20 +- .../segment/SegmentZKMetadataUtils.java} | 141 +++++---- .../TableLLCSegmentUploadResponse.java | 6 +- .../controller/api/upload/ZKOperator.java | 14 +- .../helix/core/PinotHelixResourceManager.java | 14 +- .../PinotLLCRealtimeSegmentManager.java | 268 ++++++------------ .../PinotLLCRealtimeSegmentManagerTest.java | 19 +- .../server/api/resources/TablesResource.java | 207 ++++++-------- 8 files changed, 292 insertions(+), 397 deletions(-) rename pinot-common/src/main/java/org/apache/pinot/common/{utils/helix/ZKMetadataUtils.java => metadata/segment/SegmentZKMetadataUtils.java} (54%) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java index 6373beb1cf39..9474007fc511 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadata.java @@ -64,6 +64,16 @@ public String getSegmentName() { return _znRecord.getId(); } + public Map getSimpleFields() { + return _simpleFields; + } + + public void setSimpleFields(Map simpleFields) { + _simpleFields = simpleFields; + _startTimeMsCached = false; + _endTimeMsCached = false; + } + public long getStartTimeMs() { if (!_startTimeMsCached) { long startTimeMs = -1; @@ -395,16 +405,6 @@ public static SegmentZKMetadata fromJsonString(String jsonString) return new SegmentZKMetadata(znRecord); } - public void copySimpleFieldsFrom(SegmentZKMetadata segmentZKMetadata) { - ZNRecord znRecord = segmentZKMetadata.toZNRecord(); - _simpleFields.clear(); - _simpleFields.putAll(znRecord.getSimpleFields()); - - // Reset the cached values - _startTimeMsCached = false; - _endTimeMsCached = false; - } - @Override public ZNRecord toZNRecord() { // Convert to TreeMap to keep the keys sorted. The de-serialized ZNRecord has simple fields stored as LinkedHashMap. diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java similarity index 54% rename from pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java rename to pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java index f4f1e0d42b1e..58de48a843c2 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/helix/ZKMetadataUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.common.utils.helix; +package org.apache.pinot.common.metadata.segment; import com.google.common.base.Preconditions; import java.util.HashMap; @@ -24,20 +24,20 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; -import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.SegmentMetadata; +import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.joda.time.Interval; -public class ZKMetadataUtils { - private ZKMetadataUtils() { +public class SegmentZKMetadataUtils { + private SegmentZKMetadataUtils() { } /** @@ -47,7 +47,7 @@ public static SegmentZKMetadata createSegmentZKMetadata(String tableNameWithType String downloadUrl, @Nullable String crypterName, long segmentSizeInBytes) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segmentMetadata.getName()); updateSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, downloadUrl, crypterName, - segmentSizeInBytes, true); + segmentSizeInBytes, null, true); return segmentZKMetadata; } @@ -57,7 +57,16 @@ public static SegmentZKMetadata createSegmentZKMetadata(String tableNameWithType public static void refreshSegmentZKMetadata(String tableNameWithType, SegmentZKMetadata segmentZKMetadata, SegmentMetadata segmentMetadata, String downloadUrl, @Nullable String crypterName, long segmentSizeInBytes) { updateSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, downloadUrl, crypterName, - segmentSizeInBytes, false); + segmentSizeInBytes, null, false); + } + + /** + * Updates the segment ZK metadata for a committing segment. + */ + public static void updateCommittingSegmentZKMetadata(String realtimeTableName, SegmentZKMetadata segmentZKMetadata, + SegmentMetadata segmentMetadata, String downloadUrl, long segmentSizeInBytes, String endOffset) { + updateSegmentZKMetadata(realtimeTableName, segmentZKMetadata, segmentMetadata, downloadUrl, null, + segmentSizeInBytes, endOffset, false); } public static void updateSegmentZKTimeInterval(SegmentZKMetadata segmentZKMetadata, @@ -77,47 +86,96 @@ public static void updateSegmentZKTimeInterval(SegmentZKMetadata segmentZKMetada private static void updateSegmentZKMetadata(String tableNameWithType, SegmentZKMetadata segmentZKMetadata, SegmentMetadata segmentMetadata, String downloadUrl, @Nullable String crypterName, long segmentSizeInBytes, - boolean newSegment) { - if (newSegment) { - segmentZKMetadata.setPushTime(System.currentTimeMillis()); - } else { - segmentZKMetadata.setRefreshTime(System.currentTimeMillis()); - } + @Nullable String endOffset, boolean newSegment) { + String segmentName = segmentZKMetadata.getSegmentName(); - if (segmentMetadata.getTimeInterval() != null) { - segmentZKMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis()); - segmentZKMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis()); - segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); - ColumnMetadata timeColumnMetadata = segmentMetadata.getColumnMetadataFor(segmentMetadata.getTimeColumn()); - if (isValidTimeMetadata(timeColumnMetadata)) { - segmentZKMetadata.setRawStartTime(timeColumnMetadata.getMinValue().toString()); - segmentZKMetadata.setRawEndTime(timeColumnMetadata.getMaxValue().toString()); + if (endOffset != null) { + // For committing segment + + segmentZKMetadata.setEndOffset(endOffset); + segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); + + // For committing segment, use current time as start/end time if total docs is 0 + if (segmentMetadata.getTotalDocs() > 0) { + Interval timeInterval = segmentMetadata.getTimeInterval(); + Preconditions.checkState(timeInterval != null, "Failed to find time info for table: %s, segment: %s", + tableNameWithType, segmentName); + segmentZKMetadata.setStartTime(timeInterval.getStartMillis()); + segmentZKMetadata.setEndTime(timeInterval.getEndMillis()); + } else { + long now = System.currentTimeMillis(); + segmentZKMetadata.setStartTime(now); + segmentZKMetadata.setEndTime(now); } + segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); } else { - segmentZKMetadata.setStartTime(-1); - segmentZKMetadata.setEndTime(-1); - segmentZKMetadata.setTimeUnit(null); + // For uploaded segment + + // Set segment status, start/end offset info for real-time table + if (TableNameBuilder.isRealtimeTableResource(tableNameWithType)) { + segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.UPLOADED); + + // For new segment, start/end offset must exist if the segment name follows LLC segment name convention + if (newSegment && LLCSegmentName.isLLCSegment(segmentMetadata.getName())) { + Preconditions.checkArgument( + segmentMetadata.getStartOffset() != null && segmentMetadata.getEndOffset() != null, + "New uploaded LLC segment must have start/end offset in the segment metadata"); + } + + // NOTE: + // - If start/end offset is available in the uploaded segment, update them in the segment ZK metadata + // - If not, keep the existing start/end offset in the segment ZK metadata unchanged + if (segmentMetadata.getStartOffset() != null) { + segmentZKMetadata.setStartOffset(segmentMetadata.getStartOffset()); + } + if (segmentMetadata.getEndOffset() != null) { + segmentZKMetadata.setEndOffset(segmentMetadata.getEndOffset()); + } + } + + segmentZKMetadata.setCreationTime(segmentMetadata.getIndexCreationTime()); + if (newSegment) { + segmentZKMetadata.setPushTime(System.currentTimeMillis()); + } else { + segmentZKMetadata.setRefreshTime(System.currentTimeMillis()); + } + Interval timeInterval = segmentMetadata.getTimeInterval(); + if (timeInterval != null) { + segmentZKMetadata.setStartTime(timeInterval.getStartMillis()); + segmentZKMetadata.setEndTime(timeInterval.getEndMillis()); + segmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); + ColumnMetadata timeColumnMetadata = segmentMetadata.getColumnMetadataFor(segmentMetadata.getTimeColumn()); + if (isValidTimeMetadata(timeColumnMetadata)) { + segmentZKMetadata.setRawStartTime(timeColumnMetadata.getMinValue().toString()); + segmentZKMetadata.setRawEndTime(timeColumnMetadata.getMaxValue().toString()); + } + } else { + segmentZKMetadata.setStartTime(-1); + segmentZKMetadata.setEndTime(-1); + segmentZKMetadata.setTimeUnit(null); + } } - segmentZKMetadata.setIndexVersion( - segmentMetadata.getVersion() != null ? segmentMetadata.getVersion().name() : null); + + SegmentVersion segmentVersion = segmentMetadata.getVersion(); + segmentZKMetadata.setIndexVersion(segmentVersion != null ? segmentVersion.toString() : null); segmentZKMetadata.setTotalDocs(segmentMetadata.getTotalDocs()); - segmentZKMetadata.setSizeInBytes(segmentSizeInBytes); segmentZKMetadata.setCrc(Long.parseLong(segmentMetadata.getCrc())); - segmentZKMetadata.setCreationTime(segmentMetadata.getIndexCreationTime()); segmentZKMetadata.setDownloadUrl(downloadUrl); segmentZKMetadata.setCrypterName(crypterName); + segmentZKMetadata.setSizeInBytes(segmentSizeInBytes); // Set partition metadata Map columnPartitionMap = new HashMap<>(); - segmentMetadata.getColumnMetadataMap().forEach((column, columnMetadata) -> { + for (Map.Entry entry : segmentMetadata.getColumnMetadataMap().entrySet()) { + ColumnMetadata columnMetadata = entry.getValue(); PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); if (partitionFunction != null) { ColumnPartitionMetadata columnPartitionMetadata = new ColumnPartitionMetadata(partitionFunction.getName(), partitionFunction.getNumPartitions(), columnMetadata.getPartitions(), partitionFunction.getFunctionConfig()); - columnPartitionMap.put(column, columnPartitionMetadata); + columnPartitionMap.put(entry.getKey(), columnPartitionMetadata); } - }); + } segmentZKMetadata.setPartitionMetadata( !columnPartitionMap.isEmpty() ? new SegmentPartitionMetadata(columnPartitionMap) : null); @@ -130,27 +188,6 @@ private static void updateSegmentZKMetadata(String tableNameWithType, SegmentZKM customMap.putAll(segmentMetadata.getCustomMap()); } segmentZKMetadata.setCustomMap(customMap); - - // Set fields specific to realtime table - if (TableNameBuilder.isRealtimeTableResource(tableNameWithType)) { - segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.UPLOADED); - - // For new segment, start/end offset must exist if the segment name follows LLC segment name convention - if (newSegment && LLCSegmentName.isLLCSegment(segmentMetadata.getName())) { - Preconditions.checkArgument(segmentMetadata.getStartOffset() != null && segmentMetadata.getEndOffset() != null, - "New uploaded LLC segment must have start/end offset in the segment metadata"); - } - - // NOTE: - // - If start/end offset is available in the uploaded segment, update them in the segment ZK metadata - // - If not, keep the existing start/end offset in the segment ZK metadata unchanged - if (segmentMetadata.getStartOffset() != null) { - segmentZKMetadata.setStartOffset(segmentMetadata.getStartOffset()); - } - if (segmentMetadata.getEndOffset() != null) { - segmentZKMetadata.setEndOffset(segmentMetadata.getEndOffset()); - } - } } private static boolean isValidTimeMetadata(ColumnMetadata timeColumnMetadata) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableLLCSegmentUploadResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableLLCSegmentUploadResponse.java index aa3dbe5c0bed..2bbbb52f464f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableLLCSegmentUploadResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableLLCSegmentUploadResponse.java @@ -26,8 +26,8 @@ public class TableLLCSegmentUploadResponse { private final long _crc; private final String _downloadUrl; - public TableLLCSegmentUploadResponse(@JsonProperty("segmentName") String segmentName, - @JsonProperty("crc") Long crc, @JsonProperty("downloadUrl") String downloadUrl) { + public TableLLCSegmentUploadResponse(@JsonProperty("segmentName") String segmentName, @JsonProperty("crc") long crc, + @JsonProperty("downloadUrl") String downloadUrl) { _segmentName = segmentName; _crc = crc; _downloadUrl = downloadUrl; @@ -37,7 +37,7 @@ public String getSegmentName() { return _segmentName; } - public Long getCrc() { + public long getCrc() { return _crc; } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index 73049506ba1b..4da9c72a5665 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -33,11 +33,11 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.FileUploadDownloadClient.FileUploadType; -import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.exception.ControllerApplicationException; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; @@ -309,11 +309,11 @@ private void processExistingSegment(String tableNameWithType, SegmentMetadata se if (customMapModifier == null) { // If no modifier is provided, use the custom map from the segment metadata segmentZKMetadata.setCustomMap(null); - ZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, + SegmentZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); } else { // If modifier is provided, first set the custom map from the segment metadata, then apply the modifier - ZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, + SegmentZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); segmentZKMetadata.setCustomMap(customMapModifier.modifyMap(segmentZKMetadata.getCustomMap())); } @@ -446,11 +446,11 @@ private void processExistingSegments(String tableNameWithType, FileUploadType up if (customMapModifier == null) { // If no modifier is provided, use the custom map from the segment metadata segmentZKMetadata.setCustomMap(null); - ZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, + SegmentZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); } else { // If modifier is provided, first set the custom map from the segment metadata, then apply the modifier - ZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, + SegmentZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); segmentZKMetadata.setCustomMap(customMapModifier.modifyMap(segmentZKMetadata.getCustomMap())); } @@ -502,7 +502,7 @@ private void processNewSegment(String tableNameWithType, SegmentMetadata segment SegmentZKMetadata newSegmentZKMetadata; try { newSegmentZKMetadata = - ZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, segmentDownloadURIStr, + SegmentZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); } catch (IllegalArgumentException e) { throw new ControllerApplicationException(LOGGER, @@ -590,7 +590,7 @@ private void processNewSegments(String tableNameWithType, FileUploadType uploadT long segmentSizeInBytes = segmentUploadMetadata.getSegmentSizeInBytes(); File segmentFile = segmentUploadMetadata.getEncryptionInfo().getRight(); try { - newSegmentZKMetadata = ZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, + newSegmentZKMetadata = SegmentZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, segmentDownloadURIStr, crypterName, segmentSizeInBytes); segmentZKMetadataMap.put(segmentName, newSegmentZKMetadata); segmentNames.add(segmentName); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 51724fa1a345..1f983ce6718d 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -114,6 +114,7 @@ import org.apache.pinot.common.metadata.controllerjob.ControllerJobType; import org.apache.pinot.common.metadata.instance.InstanceZKMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.minion.MinionTaskMetadataUtils; import org.apache.pinot.common.restlet.resources.EndReplaceSegmentsRequest; @@ -132,7 +133,6 @@ import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.helix.HelixHelper; import org.apache.pinot.common.utils.helix.PinotHelixPropertyStoreZnRecordProvider; -import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.exception.ControllerApplicationException; import org.apache.pinot.controller.api.exception.InvalidTableConfigException; @@ -2338,9 +2338,9 @@ public boolean updateJobsForTable(String tableNameWithType, String jobType, Cons public void addNewSegment(String tableNameWithType, SegmentMetadata segmentMetadata, String downloadUrl) { // NOTE: must first set the segment ZK metadata before assigning segment to instances because segment assignment // might need them to determine the partition of the segment, and server will need them to download the segment - SegmentZKMetadata segmentZkmetadata = - ZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, downloadUrl, null, -1); - ZNRecord znRecord = segmentZkmetadata.toZNRecord(); + SegmentZKMetadata segmentZKMetadata = + SegmentZKMetadataUtils.createSegmentZKMetadata(tableNameWithType, segmentMetadata, downloadUrl, null, -1); + ZNRecord znRecord = segmentZKMetadata.toZNRecord(); String segmentName = segmentMetadata.getName(); String segmentZKMetadataPath = @@ -2598,7 +2598,7 @@ private void deleteTableOnServers(String tableNameWithType) { } public void updateZkTimeInterval(SegmentZKMetadata segmentZKMetadata, DateTimeFieldSpec timeColumnFieldSpec) { - ZKMetadataUtils.updateSegmentZKTimeInterval(segmentZKMetadata, timeColumnFieldSpec); + SegmentZKMetadataUtils.updateSegmentZKTimeInterval(segmentZKMetadata, timeColumnFieldSpec); } @VisibleForTesting @@ -2610,8 +2610,8 @@ public void refreshSegment(String tableNameWithType, SegmentMetadata segmentMeta // ZK metadata to refresh the segment (server will compare the segment ZK metadata with the local metadata to decide // whether to download the new segment; broker will update the segment partition info & time boundary based on the // segment ZK metadata) - ZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, downloadUrl, null, - -1); + SegmentZKMetadataUtils.refreshSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, downloadUrl, + null, -1); if (!ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata, expectedVersion)) { throw new RuntimeException( diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 9629199a6573..7f71b8887bb8 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -64,6 +64,7 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.metrics.ControllerGauge; import org.apache.pinot.common.metrics.ControllerMeter; import org.apache.pinot.common.metrics.ControllerMetrics; @@ -91,10 +92,7 @@ import org.apache.pinot.controller.validation.RealtimeSegmentValidationManager; import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.core.util.PeerServerSegmentFinder; -import org.apache.pinot.segment.spi.ColumnMetadata; -import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; -import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.PauseState; @@ -448,10 +446,6 @@ public IdealState getIdealState(String realtimeTableName) { } } - public ExternalView getExternalView(String realtimeTableName) { - return _helixResourceManager.getTableExternalView(realtimeTableName); - } - @VisibleForTesting void setIdealState(String realtimeTableName, IdealState idealState) { try { @@ -540,7 +534,7 @@ private void commitSegmentMetadataInternal(String realtimeTableName, /* * Update zookeeper in 3 steps. * - * Step 1: Update PROPERTYSTORE to change the old segment metadata status to COMMITTING/ DONE + * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE/COMMITTING * Step 2: Update PROPERTYSTORE to create the new segment metadata with status IN_PROGRESS * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state. */ @@ -630,8 +624,8 @@ private SegmentZKMetadata updateCommittingSegmentMetadata(String realtimeTableNa // Step 2: Create new segment metadata private String createNewSegmentMetadata(TableConfig tableConfig, IdealState idealState, - CommittingSegmentDescriptor committingSegmentDescriptor, - SegmentZKMetadata committingSegmentZKMetadata, InstancePartitions instancePartitions) { + CommittingSegmentDescriptor committingSegmentDescriptor, SegmentZKMetadata committingSegmentZKMetadata, + InstancePartitions instancePartitions) { String committingSegmentName = committingSegmentDescriptor.getSegmentName(); String realtimeTableName = tableConfig.getTableName(); @@ -642,9 +636,10 @@ private String createNewSegmentMetadata(TableConfig tableConfig, IdealState idea LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); - List streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig).stream().map( - streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig) - ).collect(Collectors.toList()); + List streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig) + .stream() + .map(streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig)) + .collect(Collectors.toList()); Set partitionIds = getPartitionIds(streamConfigs, idealState); if (partitionIds.contains(committingSegmentPartitionGroupId)) { @@ -708,12 +703,12 @@ public void commitSegmentStartMetadata(String realtimeTableName, */ public void commitSegmentEndMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { - // Used to inject failure for testing. RealtimeSegmentValidationManager should be able to fix the - // segment that encounter failure at this stage of commit protocol. - preProcessCommitSegmentEndMetadata(); Preconditions.checkState(!_isStopping, "Segment manager is stopping"); + + preProcessCommitSegmentEndMetadata(); + + _numCompletingSegments.addAndGet(1); try { - _numCompletingSegments.addAndGet(1); // Validate segment location only for metadata commit if (StringUtils.isBlank(committingSegmentDescriptor.getSegmentLocation())) { LOGGER.warn("Committing segment: {} was not uploaded to deep store", @@ -760,7 +755,6 @@ private SegmentZKMetadata updateCommittingSegmentZKMetadataToCOMMITTING(String r "Segment status for segment: %s should be IN_PROGRESS, found: %s", segmentName, committingSegmentZKMetadata.getStatus()); - // TODO Issue 5953 remove the long parsing once metadata is set correctly. committingSegmentZKMetadata.setEndOffset(committingSegmentDescriptor.getNextOffset()); committingSegmentZKMetadata.setStatus(Status.COMMITTING); @@ -768,7 +762,6 @@ private SegmentZKMetadata updateCommittingSegmentZKMetadataToCOMMITTING(String r return committingSegmentZKMetadata; } - /** * Updates segment ZK metadata for the committing segment. */ @@ -786,38 +779,12 @@ private SegmentZKMetadata updateCommittingSegmentZKMetadata(String realtimeTable Preconditions.checkState(segmentMetadata != null, "Failed to find segment metadata from descriptor for segment: %s", segmentName); - // TODO Issue 5953 remove the long parsing once metadata is set correctly. - committingSegmentZKMetadata.setEndOffset(committingSegmentDescriptor.getNextOffset()); - committingSegmentZKMetadata.setStatus(Status.DONE); - // If the download url set by the server is a peer download url format with peer scheme, put - // METADATA_URI_FOR_PEER_DOWNLOAD in zk; otherwise just use the location in the descriptor. - committingSegmentZKMetadata.setDownloadUrl(isPeerURL(committingSegmentDescriptor.getSegmentLocation()) - ? CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD : committingSegmentDescriptor.getSegmentLocation()); - committingSegmentZKMetadata.setCrc(Long.valueOf(segmentMetadata.getCrc())); - if (segmentMetadata.getTotalDocs() > 0) { - Preconditions.checkNotNull(segmentMetadata.getTimeInterval(), - "start/end time information is not correctly written to the segment for table: " + realtimeTableName); - committingSegmentZKMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis()); - committingSegmentZKMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis()); - } else { - // Set current time as start/end time if total docs is 0 - long now = System.currentTimeMillis(); - committingSegmentZKMetadata.setStartTime(now); - committingSegmentZKMetadata.setEndTime(now); - } - committingSegmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS); - SegmentVersion segmentVersion = segmentMetadata.getVersion(); - if (segmentVersion != null) { - committingSegmentZKMetadata.setIndexVersion(segmentVersion.name()); - } - committingSegmentZKMetadata.setTotalDocs(segmentMetadata.getTotalDocs()); - committingSegmentZKMetadata.setSizeInBytes(committingSegmentDescriptor.getSegmentSizeBytes()); - - // Update the partition group metadata based on the segment metadata - // NOTE: When the stream partition changes, or the records are not properly partitioned from the stream, the - // partition of the segment (based on the actual consumed records) can be different from the stream partition. - committingSegmentZKMetadata.setPartitionMetadata(getPartitionMetadataFromSegmentMetadata(segmentMetadata)); - + String segmentLocation = committingSegmentDescriptor.getSegmentLocation(); + String downloadUrl = + isPeerURL(segmentLocation) ? CommonConstants.Segment.METADATA_URI_FOR_PEER_DOWNLOAD : segmentLocation; + SegmentZKMetadataUtils.updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentZKMetadata, + segmentMetadata, downloadUrl, committingSegmentDescriptor.getSegmentSizeBytes(), + committingSegmentDescriptor.getNextOffset()); persistSegmentZKMetadata(realtimeTableName, committingSegmentZKMetadata, stat.getVersion()); return committingSegmentZKMetadata; } @@ -893,22 +860,6 @@ private SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig } } - @Nullable - private SegmentPartitionMetadata getPartitionMetadataFromSegmentMetadata(SegmentMetadataImpl segmentMetadata) { - for (Map.Entry entry : segmentMetadata.getColumnMetadataMap().entrySet()) { - // NOTE: There is at most one partition column. - ColumnMetadata columnMetadata = entry.getValue(); - PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); - if (partitionFunction != null) { - ColumnPartitionMetadata columnPartitionMetadata = - new ColumnPartitionMetadata(partitionFunction.getName(), partitionFunction.getNumPartitions(), - columnMetadata.getPartitions(), columnMetadata.getPartitionFunction().getFunctionConfig()); - return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); - } - } - return null; - } - public long getCommitTimeoutMS(String realtimeTableName) { long commitTimeoutMS = SegmentCompletionProtocol.getMaxSegmentCommitTimeMs(); if (_propertyStore == null) { @@ -1078,7 +1029,7 @@ private Map getLatestSegmentZKMetadataMap(String rea * leader of the table. * * During segment commit, we update zookeeper in 3 steps - * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE/ COMMITTING + * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE/COMMITTING * Step 2: Update PROPERTYSTORE to create the new segment metadata with status IN_PROGRESS * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state. * @@ -1381,7 +1332,7 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, List uploadAttempts = Arrays.asList( - // Primary method - () -> { - String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegmentToDeepStore"); - LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, - serverUploadRequestUrl); - SegmentZKMetadata uploadedMetadata = _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata( - serverUploadRequestUrl); - handleMetadataUpload(segmentName, rawTableName, segmentZKMetadata, uploadedMetadata, pinotFS); - }, - // First fallback - () -> { - String serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegment"); - LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, - serverUploadRequestUrl); - TableLLCSegmentUploadResponse response = - _fileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl); - handleLLCUpload(segmentName, rawTableName, segmentZKMetadata, response, pinotFS); - }, - // Legacy fallback - () -> { - String serverUploadRequestUrl = getUploadUrl(uri, "upload"); - LOGGER.info("Ask server to upload LLC segment {} to deep store by this path: {}", segmentName, - serverUploadRequestUrl); - - String tempUrl = _fileUploadDownloadClient.uploadToSegmentStore(serverUploadRequestUrl); - handleBasicUpload(segmentName, rawTableName, segmentZKMetadata, tempUrl, pinotFS); - } - ); - - // Try each method in sequence until one succeeds - Exception lastException = null; - for (UploadAttempt attempt : uploadAttempts) { - try { - attempt.upload(); - return; // Success, exit the method - } catch (Exception e) { - lastException = e; - LOGGER.warn("Upload attempt failed for segment {}, trying next method", segmentName, e); + SegmentZKMetadata segmentZKMetadata, PinotFS pinotFS) { + String serverUploadRequestUrl = getUploadUrl(uri, "uploadCommittedSegment"); + LOGGER.info("Asking server to upload segment: {} by path: {}", segmentName, serverUploadRequestUrl); + try { + SegmentZKMetadata uploadedMetadata = + _fileUploadDownloadClient.uploadLLCToSegmentStoreWithZKMetadata(serverUploadRequestUrl); + handleMetadataUpload(rawTableName, segmentName, segmentZKMetadata, uploadedMetadata, pinotFS); + return; + } catch (Exception e) { + if (segmentZKMetadata.getStatus() == Status.COMMITTING) { + throw new RuntimeException("Failed to ask server to upload COMMITTING segment: " + segmentName, e); } + LOGGER.warn("Failed to ask server to upload segment: {} by path: {}, falling back to other upload methods", + segmentName, serverUploadRequestUrl, e); + } + + serverUploadRequestUrl = getUploadUrl(uri, "uploadLLCSegment"); + LOGGER.info("Asking server to upload segment: {} by path: {}", segmentName, serverUploadRequestUrl); + try { + TableLLCSegmentUploadResponse response = + _fileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl); + handleLLCUpload(segmentName, rawTableName, segmentZKMetadata, response, pinotFS); + return; + } catch (Exception e) { + LOGGER.warn("Failed to ask server to upload segment: {} by path: {}, falling back to other upload methods", + segmentName, serverUploadRequestUrl, e); } - // All attempts for segment upload failed - throw new Exception("All upload attempts failed for segment " + segmentName, lastException); + serverUploadRequestUrl = getUploadUrl(uri, "upload"); + LOGGER.info("Asking server to upload segment: {} by path: {}", segmentName, serverUploadRequestUrl); + try { + String segmentLocation = _fileUploadDownloadClient.uploadToSegmentStore(serverUploadRequestUrl); + handleBasicUpload(rawTableName, segmentName, segmentZKMetadata, segmentLocation, pinotFS); + } catch (Exception e) { + throw new RuntimeException("Failed to ask server to upload segment: " + segmentName, e); + } } private String getUploadUrl(URI uri, String endpoint) { - return String.format("%s/%s?uploadTimeoutMs=%d", - uri.toString(), endpoint, _deepstoreUploadRetryTimeoutMs); + return String.format("%s/%s?uploadTimeoutMs=%d", uri.toString(), endpoint, _deepstoreUploadRetryTimeoutMs); } - private void handleMetadataUpload(String segmentName, String rawTableName, SegmentZKMetadata currentMetadata, - SegmentZKMetadata uploadedMetadata, - PinotFS pinotFS) + private void handleMetadataUpload(String rawTableName, String segmentName, SegmentZKMetadata currentMetadata, + SegmentZKMetadata uploadedMetadata, PinotFS pinotFS) throws Exception { - - if (uploadedMetadata.getCrc() != currentMetadata.getCrc()) { - LOGGER.info("Updating segment {} crc in ZK to be {} from previous {}", segmentName, - uploadedMetadata.getCrc(), currentMetadata.getCrc()); - updateSegmentMetadata(currentMetadata, uploadedMetadata); + if (currentMetadata.getStatus() == Status.COMMITTING) { + LOGGER.info("Updating ZK metadata for committing segment: {}", segmentName); + currentMetadata.setSimpleFields(uploadedMetadata.getSimpleFields()); + } else if (currentMetadata.getCrc() != uploadedMetadata.getCrc()) { + LOGGER.info("Updating CRC in ZK metadata for segment: {} from: {} to: {}", segmentName, currentMetadata.getCrc(), + uploadedMetadata.getCrc()); + currentMetadata.setCrc(uploadedMetadata.getCrc()); } - - String downloadUrl = moveSegmentFile(rawTableName, segmentName, uploadedMetadata.getDownloadUrl(), pinotFS); - LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); - currentMetadata.setDownloadUrl(downloadUrl); + moveSegmentAndSetDownloadUrl(rawTableName, segmentName, uploadedMetadata.getDownloadUrl(), pinotFS, + currentMetadata); } - private void handleLLCUpload(String segmentName, String rawTableName, - SegmentZKMetadata currentMetadata, TableLLCSegmentUploadResponse response, - PinotFS pinotFS) + private void moveSegmentAndSetDownloadUrl(String rawTableName, String segmentName, String segmentLocation, + PinotFS pinotFS, SegmentZKMetadata segmentZKMetadata) throws Exception { - - String downloadUrl = moveSegmentFile(rawTableName, segmentName, response.getDownloadUrl(), pinotFS); - LOGGER.info("Updating segment {} download url in ZK to be {}", segmentName, downloadUrl); - currentMetadata.setDownloadUrl(downloadUrl); - - if (response.getCrc() != currentMetadata.getCrc()) { - LOGGER.info("Updating segment {} crc in ZK to be {} from previous {}", segmentName, - response.getCrc(), currentMetadata.getCrc()); - currentMetadata.setCrc(response.getCrc()); - } + String newDownloadUrl = moveSegmentFile(rawTableName, segmentName, segmentLocation, pinotFS); + LOGGER.info("Updating download url in ZK metadata for segment: {} to: {}", segmentName, newDownloadUrl); + segmentZKMetadata.setDownloadUrl(newDownloadUrl); } - private void handleBasicUpload(String segmentName, String rawTableName, - SegmentZKMetadata metadata, String tempDownloadUrl, PinotFS pinotFS) + private void handleLLCUpload(String segmentName, String rawTableName, SegmentZKMetadata currentMetadata, + TableLLCSegmentUploadResponse response, PinotFS pinotFS) throws Exception { - - String downloadUrl = moveSegmentFile(rawTableName, segmentName, tempDownloadUrl, pinotFS); - metadata.setDownloadUrl(downloadUrl); + long currentCrc = currentMetadata.getCrc(); + long newCrc = response.getCrc(); + if (currentCrc != newCrc) { + LOGGER.info("Updating CRC in ZK metadata for segment: {} from: {} to: {}", segmentName, currentCrc, newCrc); + currentMetadata.setCrc(newCrc); + } + moveSegmentAndSetDownloadUrl(rawTableName, segmentName, response.getDownloadUrl(), pinotFS, currentMetadata); } - /** - * Updates the segment metadata in ZooKeeper with information from the uploaded segment. - * - * For pauseless consumption scenarios: - * - When segment status is COMMITTING, it indicates a previous segment commit metadata update failed - * - In this case, we perform a full metadata update including time boundaries, index details, and partition info - * - Finally, the segment status is marked as DONE to indicate successful completion - * - * For regular consumption: - * - Only the CRC value is updated - * - * @param segmentZKMetadata Current segment metadata stored in ZooKeeper that needs to be updated - * @param uploadedSegmentZKMetadata New metadata from the successfully uploaded segment - */ - private void updateSegmentMetadata(SegmentZKMetadata segmentZKMetadata, SegmentZKMetadata uploadedSegmentZKMetadata) { - if (segmentZKMetadata.getStatus() == Status.COMMITTING) { - LOGGER.info("Updating additional metadata in ZK for segment {} as pauseless is enabled", - segmentZKMetadata.getSegmentName()); - // Copy all the simple fields from the uploaded segment - segmentZKMetadata.copySimpleFieldsFrom(uploadedSegmentZKMetadata); - } - segmentZKMetadata.setCrc(uploadedSegmentZKMetadata.getCrc()); + private void handleBasicUpload(String rawTableName, String segmentName, SegmentZKMetadata metadata, + String segmentLocation, PinotFS pinotFS) + throws Exception { + moveSegmentAndSetDownloadUrl(rawTableName, segmentName, segmentLocation, pinotFS, metadata); } - private boolean shouldSkipSegmentForDeepStoreUpload(TableConfig tableConfig, - SegmentZKMetadata segmentZKMetadata, RetentionStrategy retentionStrategy) { + private boolean shouldSkipSegmentForDeepStoreUpload(TableConfig tableConfig, SegmentZKMetadata segmentZKMetadata, + RetentionStrategy retentionStrategy) { String realtimeTableName = tableConfig.getTableName(); String segmentName = segmentZKMetadata.getSegmentName(); diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 18710107dd96..13289864d86b 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -210,8 +210,9 @@ public void testCommitSegment() { // Commit a segment for partition group 0 String committingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName(); - CommittingSegmentDescriptor committingSegmentDescriptor = new CommittingSegmentDescriptor(committingSegment, - new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString(), SEGMENT_SIZE_IN_BYTES); + String nextOffset = new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString(); + CommittingSegmentDescriptor committingSegmentDescriptor = + new CommittingSegmentDescriptor(committingSegment, nextOffset, SEGMENT_SIZE_IN_BYTES); committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata()); segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor); @@ -231,8 +232,7 @@ public void testCommitSegment() { SegmentZKMetadata committedSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(committingSegment); assertEquals(committedSegmentZKMetadata.getStatus(), Status.DONE); assertEquals(committedSegmentZKMetadata.getStartOffset(), PARTITION_OFFSET.toString()); - assertEquals(committedSegmentZKMetadata.getEndOffset(), - new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString()); + assertEquals(committedSegmentZKMetadata.getEndOffset(), nextOffset); assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); assertEquals(committedSegmentZKMetadata.getCrc(), Long.parseLong(CRC)); assertEquals(committedSegmentZKMetadata.getIndexVersion(), SEGMENT_VERSION.name()); @@ -241,8 +241,7 @@ public void testCommitSegment() { SegmentZKMetadata consumingSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(consumingSegment); assertEquals(consumingSegmentZKMetadata.getStatus(), Status.IN_PROGRESS); - assertEquals(consumingSegmentZKMetadata.getStartOffset(), - new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString()); + assertEquals(consumingSegmentZKMetadata.getStartOffset(), nextOffset); assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); // Turn one instance of the consuming segment OFFLINE and commit the segment @@ -1182,7 +1181,7 @@ public void testUploadToSegmentStoreV2() } @Test - public void testUploadLLCSegmentToDeepStore() + public void testUploadCommittedSegment() throws HttpErrorStatusException, IOException, URISyntaxException { // mock the behavior for PinotHelixResourceManager PinotHelixResourceManager pinotHelixResourceManager = mock(PinotHelixResourceManager.class); @@ -1232,8 +1231,7 @@ public void testUploadLLCSegmentToDeepStore() when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance0)).thenReturn(instanceConfig0); // mock the request/response for 1st segment upload String serverUploadRequestUrl0 = - String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance0, - adminPort, + String.format("http://%s:%d/segments/%s/%s/uploadCommittedSegment?uploadTimeoutMs=-1", instance0, adminPort, REALTIME_TABLE_NAME, segmentsZKMetadata.get(0).getSegmentName()); // tempSegmentFileLocation is the location where the segment uploader will upload the segment. This usually ends // with a random UUID @@ -1263,8 +1261,7 @@ public void testUploadLLCSegmentToDeepStore() when(helixAdmin.getInstanceConfig(CLUSTER_NAME, instance1)).thenReturn(instanceConfig1); // mock the request/response for 2nd segment upload String serverUploadRequestUrl1 = - String.format("http://%s:%d/segments/%s/%s/uploadLLCSegmentToDeepStore?uploadTimeoutMs=-1", instance1, - adminPort, + String.format("http://%s:%d/segments/%s/%s/uploadCommittedSegment?uploadTimeoutMs=-1", instance1, adminPort, REALTIME_TABLE_NAME, segmentsZKMetadata.get(1).getSegmentName()); when(segmentManager._mockedFileUploadDownloadClient.uploadLLCToSegmentStore(serverUploadRequestUrl1)).thenThrow( new HttpErrorStatusException("failed to upload segment", diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index a3bfd661adef..f7038ed11f4e 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -29,6 +29,7 @@ import io.swagger.annotations.SecurityDefinition; import io.swagger.annotations.SwaggerDefinition; import java.io.File; +import java.io.IOException; import java.net.URI; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -61,6 +62,7 @@ import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataUtils; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; @@ -78,7 +80,6 @@ import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.helix.HelixHelper; -import org.apache.pinot.common.utils.helix.ZKMetadataUtils; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; import org.apache.pinot.core.data.manager.realtime.RealtimeSegmentDataManager; @@ -104,7 +105,6 @@ import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.ConsumerPartitionState; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -825,33 +825,8 @@ public String uploadLLCSegment( File segmentTarFile = null; try { - // Create the tar.gz segment file in the server's segmentTarUploadDir folder with a unique file name. - File segmentTarUploadDir = - new File(_serverInstance.getInstanceDataManager().getSegmentFileDirectory(), SEGMENT_UPLOAD_DIR); - segmentTarUploadDir.mkdir(); - - segmentTarFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(segmentTarUploadDir, - tableNameWithType + "_" + segmentName + "_" + UUID.randomUUID() + TarCompressionUtils.TAR_GZ_FILE_EXTENSION, - "Invalid table / segment name: %s, %s", tableNameWithType, segmentName); - - TarCompressionUtils.createCompressedTarFile(new File(tableDataManager.getTableDataDir(), segmentName), - segmentTarFile); - - // Use segment uploader to upload the segment tar file to segment store and return the segment download url. - SegmentUploader segmentUploader = _serverInstance.getInstanceDataManager().getSegmentUploader(); - URI segmentDownloadUrl; - if (timeoutMs <= 0) { - // Use default timeout if passed timeout is not positive - segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName)); - } else { - segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName), timeoutMs); - } - if (segmentDownloadUrl == null) { - throw new WebApplicationException( - String.format("Failed to upload table %s segment %s to segment store", realtimeTableName, segmentName), - Response.Status.INTERNAL_SERVER_ERROR); - } - return segmentDownloadUrl.toString(); + segmentTarFile = createSegmentTarFile(tableDataManager, segmentName); + return uploadSegment(segmentTarFile, tableNameWithType, segmentName, timeoutMs); } finally { FileUtils.deleteQuietly(segmentTarFile); tableDataManager.releaseSegment(segmentDataManager); @@ -873,6 +848,7 @@ public String uploadLLCSegment( * @return full url where the segment is uploaded, crc, segmentName. Can add more segment metadata in the future. * @throws Exception if an error occurred during the segment upload. */ + @Deprecated @POST @Path("/segments/{realtimeTableNameWithType}/{segmentName}/uploadLLCSegment") @Produces(MediaType.APPLICATION_JSON) @@ -919,38 +895,13 @@ public TableLLCSegmentUploadResponse uploadLLCSegmentV2( String.format("Table %s segment %s does not exist", realtimeTableNameWithType, segmentName), Response.Status.NOT_FOUND); } - String crc = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); File segmentTarFile = null; try { - // Create the tar.gz segment file in the server's segmentTarUploadDir folder with a unique file name. - File segmentTarUploadDir = - new File(_serverInstance.getInstanceDataManager().getSegmentFileDirectory(), SEGMENT_UPLOAD_DIR); - segmentTarUploadDir.mkdir(); - - segmentTarFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(segmentTarUploadDir, - realtimeTableNameWithType + "_" + segmentName + "_" + UUID.randomUUID() - + TarCompressionUtils.TAR_GZ_FILE_EXTENSION, "Invalid table / segment name: %s, %s", - realtimeTableNameWithType, segmentName); - - TarCompressionUtils.createCompressedTarFile(new File(tableDataManager.getTableDataDir(), segmentName), - segmentTarFile); - - // Use segment uploader to upload the segment tar file to segment store and return the segment download url. - SegmentUploader segmentUploader = _serverInstance.getInstanceDataManager().getSegmentUploader(); - URI segmentDownloadUrl; - if (timeoutMs <= 0) { - // Use default timeout if passed timeout is not positive - segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName)); - } else { - segmentDownloadUrl = segmentUploader.uploadSegment(segmentTarFile, new LLCSegmentName(segmentName), timeoutMs); - } - if (segmentDownloadUrl == null) { - throw new WebApplicationException( - String.format("Failed to upload table %s segment %s to segment store", realtimeTableNameWithType, - segmentName), Response.Status.INTERNAL_SERVER_ERROR); - } - return new TableLLCSegmentUploadResponse(segmentName, Long.parseLong(crc), segmentDownloadUrl.toString()); + segmentTarFile = createSegmentTarFile(tableDataManager, segmentName); + String downloadUrl = uploadSegment(segmentTarFile, realtimeTableNameWithType, segmentName, timeoutMs); + return new TableLLCSegmentUploadResponse(segmentName, + Long.parseLong(segmentDataManager.getSegment().getSegmentMetadata().getCrc()), downloadUrl); } finally { FileUtils.deleteQuietly(segmentTarFile); tableDataManager.releaseSegment(segmentDataManager); @@ -958,117 +909,82 @@ public TableLLCSegmentUploadResponse uploadLLCSegmentV2( } /** - * Upload a low level consumer segment to segment store and return the segment download url, crc and - * other segment metadata. This endpoint is used when segment store copy is unavailable for committed - * low level consumer segments. - * Please note that invocation of this endpoint may cause query performance to suffer, since we tar up the segment - * to upload it. + * Upload a real-time committed segment to segment store and return the segment ZK metadata in json format. + * This endpoint is used when segment store copy is unavailable for real-time committed segments. + * Please note that invocation of this endpoint may cause query performance to suffer, since we tar up the segment to + * upload it. * * @see Date: Wed, 29 Jan 2025 10:31:35 +0530 Subject: [PATCH 46/65] The tests were running slow due to the condition that the IdealState can only be updated after a fixed time has elapsed Reduced the time requirements by creating a FakePauselessLLCRealtimeSegmentManager. --- .../PinotLLCRealtimeSegmentManager.java | 5 ++- .../PinotLLCRealtimeSegmentManagerTest.java | 9 +++-- .../BasePauselessRealtimeIngestionTest.java | 38 ++++++++++++++++++- 3 files changed, 45 insertions(+), 7 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 7f71b8887bb8..f4f41ea5548c 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -406,7 +406,7 @@ private SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String } @VisibleForTesting - SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, @Nullable Stat stat) { + protected SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, @Nullable Stat stat) { try { ZNRecord znRecord = _propertyStore.get(ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, segmentName), @@ -1253,7 +1253,8 @@ private void handleSegmentMovement(String realtimeTableName, Map stat.getMtime() + MAX_SEGMENT_COMPLETION_TIME_MILLIS) { diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java index 13289864d86b..ba1d9345daf8 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java @@ -1501,7 +1501,8 @@ List getLLCSegments(String realtimeTableName) { } @Override - SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, @Nullable Stat stat) { + protected SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, + @Nullable Stat stat) { Preconditions.checkState(_segmentZKMetadataMap.containsKey(segmentName)); if (stat != null) { stat.setVersion(_segmentZKMetadataVersionMap.get(segmentName)); @@ -1561,7 +1562,8 @@ List getNewPartitionGroupMetadataList(List } @Override - boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, long currentTimeMs) { + protected boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, + long currentTimeMs) { return _exceededMaxSegmentCompletionTime; } @@ -1584,7 +1586,8 @@ enum Scenario { } @Override - SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, @Nullable Stat stat) { + protected SegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, + @Nullable Stat stat) { SegmentZKMetadata segmentZKMetadata = super.getSegmentZKMetadata(realtimeTableName, segmentName, stat); switch (_scenario) { case ZK_VERSION_CHANGED: diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index f84498332d16..5d7f7caa4a08 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -26,9 +26,11 @@ import java.util.Map; import org.apache.commons.io.FileUtils; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.controller.BaseControllerStarter; import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingControllerStarter; @@ -43,6 +45,7 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterClass; @@ -54,9 +57,9 @@ public abstract class BasePauselessRealtimeIngestionTest extends BaseClusterIntegrationTest { protected static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES protected static final long DEFAULT_COUNT_STAR_RESULT = 115545L; protected static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; + private static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 10_000; protected List _avroFiles; protected boolean _failureEnabled = false; @@ -72,7 +75,7 @@ public abstract class BasePauselessRealtimeIngestionTest extends BaseClusterInte @Override public BaseControllerStarter createControllerStarter() { - return new FailureInjectingControllerStarter(); + return new FakePauselessControllerStarter(); } @Override @@ -259,4 +262,35 @@ private boolean hasSegmentsInStatus(List segmentZKMetadataLis } return false; } + + private static class FakePauselessLLCRealtimeSegmentManager extends FailureInjectingPinotLLCRealtimeSegmentManager { + + public FakePauselessLLCRealtimeSegmentManager( + PinotHelixResourceManager helixResourceManager, + ControllerConf controllerConf, ControllerMetrics controllerMetrics) { + super(helixResourceManager, controllerConf, controllerMetrics); + } + + @Override + protected boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, + long currentTimeMs) { + Stat stat = new Stat(); + getSegmentZKMetadata(realtimeTableName, segmentName, stat); + if (currentTimeMs > stat.getMtime() + MAX_SEGMENT_COMPLETION_TIME_MILLIS) { + LOGGER.info("Segment: {} exceeds the max completion time: {}ms, metadata update time: {}, current time: {}", + segmentName, MAX_SEGMENT_COMPLETION_TIME_MILLIS, stat.getMtime(), currentTimeMs); + return true; + } else { + return false; + } + } + } + + private static class FakePauselessControllerStarter extends FailureInjectingControllerStarter { + + @Override + protected PinotLLCRealtimeSegmentManager createPinotLLCRealtimeSegmentManager() { + return new FakePauselessLLCRealtimeSegmentManager(_helixResourceManager, getConfig(), _controllerMetrics); + } + } } From 1ba5b6cbc160f434f3d3d52d87688c740f9b8e6c Mon Sep 17 00:00:00 2001 From: KKCorps Date: Wed, 29 Jan 2025 15:08:53 +0530 Subject: [PATCH 47/65] Refactor Reingestion integration test --- .../PinotLLCRealtimeSegmentManager.java | 4 ++ .../BasePauselessRealtimeIngestionTest.java | 46 ++++--------------- ...timeIngestionSegmentCommitFailureTest.java | 36 +++++++-------- ...jectingPinotLLCRealtimeSegmentManager.java | 23 ++++++++++ 4 files changed, 53 insertions(+), 56 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index f4f41ea5548c..3f7bec6f95f4 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -446,6 +446,10 @@ public IdealState getIdealState(String realtimeTableName) { } } + public ExternalView getExternalView(String realtimeTableName) { + return _helixResourceManager.getTableExternalView(realtimeTableName); + } + @VisibleForTesting void setIdealState(String realtimeTableName, IdealState idealState) { try { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java index 5d7f7caa4a08..b5ab41e55e58 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasePauselessRealtimeIngestionTest.java @@ -26,11 +26,9 @@ import java.util.Map; import org.apache.commons.io.FileUtils; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.controller.BaseControllerStarter; import org.apache.pinot.controller.ControllerConf; -import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingControllerStarter; @@ -45,7 +43,6 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; -import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterClass; @@ -75,7 +72,7 @@ public abstract class BasePauselessRealtimeIngestionTest extends BaseClusterInte @Override public BaseControllerStarter createControllerStarter() { - return new FakePauselessControllerStarter(); + return new FailureInjectingControllerStarter(); } @Override @@ -110,7 +107,7 @@ public void setUp() startController(); startBroker(); startServer(); - + setMaxSegmentCompletionTimeMillis(); setupNonPauselessTable(); injectFailure(); setupPauselessTable(); @@ -165,6 +162,14 @@ private void setupPauselessTable() addTableConfig(tableConfig); } + private void setMaxSegmentCompletionTimeMillis() { + PinotLLCRealtimeSegmentManager realtimeSegmentManager = _helixResourceManager.getRealtimeSegmentManager(); + if (realtimeSegmentManager instanceof FailureInjectingPinotLLCRealtimeSegmentManager) { + ((FailureInjectingPinotLLCRealtimeSegmentManager) realtimeSegmentManager) + .setMaxSegmentCompletionTimeoutMs(MAX_SEGMENT_COMPLETION_TIME_MILLIS); + } + } + protected void injectFailure() { PinotLLCRealtimeSegmentManager realtimeSegmentManager = _helixResourceManager.getRealtimeSegmentManager(); if (realtimeSegmentManager instanceof FailureInjectingPinotLLCRealtimeSegmentManager) { @@ -262,35 +267,4 @@ private boolean hasSegmentsInStatus(List segmentZKMetadataLis } return false; } - - private static class FakePauselessLLCRealtimeSegmentManager extends FailureInjectingPinotLLCRealtimeSegmentManager { - - public FakePauselessLLCRealtimeSegmentManager( - PinotHelixResourceManager helixResourceManager, - ControllerConf controllerConf, ControllerMetrics controllerMetrics) { - super(helixResourceManager, controllerConf, controllerMetrics); - } - - @Override - protected boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, - long currentTimeMs) { - Stat stat = new Stat(); - getSegmentZKMetadata(realtimeTableName, segmentName, stat); - if (currentTimeMs > stat.getMtime() + MAX_SEGMENT_COMPLETION_TIME_MILLIS) { - LOGGER.info("Segment: {} exceeds the max completion time: {}ms, metadata update time: {}, current time: {}", - segmentName, MAX_SEGMENT_COMPLETION_TIME_MILLIS, stat.getMtime(), currentTimeMs); - return true; - } else { - return false; - } - } - } - - private static class FakePauselessControllerStarter extends FailureInjectingControllerStarter { - - @Override - protected PinotLLCRealtimeSegmentManager createPinotLLCRealtimeSegmentManager() { - return new FakePauselessLLCRealtimeSegmentManager(_helixResourceManager, getConfig(), _controllerMetrics); - } - } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java index e37714d9976c..b4489d4512cf 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -24,18 +24,20 @@ import java.net.URISyntaxException; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.commons.io.FileUtils; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.controller.BaseControllerStarter; import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; import org.apache.pinot.controller.helix.core.realtime.SegmentCompletionConfig; +import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingControllerStarter; +import org.apache.pinot.integration.tests.realtime.utils.FailureInjectingPinotLLCRealtimeSegmentManager; import org.apache.pinot.server.starter.helix.HelixInstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; @@ -60,7 +62,7 @@ public class PauselessRealtimeIngestionSegmentCommitFailureTest extends BaseClusterIntegrationTest { private static final int NUM_REALTIME_SEGMENTS = 48; - protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES + protected static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 10_000; // 5 MINUTES private static final Logger LOGGER = LoggerFactory.getLogger(PauselessRealtimeIngestionSegmentCommitFailureTest.class); private static final String DEFAULT_TABLE_NAME_2 = DEFAULT_TABLE_NAME + "_2"; @@ -93,6 +95,11 @@ protected void overrideServerConf(PinotConfiguration serverConf) { "org.apache.pinot.integration.tests.realtime.FailureInjectingTableDataManagerProvider"); } + @Override + public BaseControllerStarter createControllerStarter() { + return new FailureInjectingControllerStarter(); + } + @BeforeClass public void setUp() throws Exception { @@ -110,6 +117,7 @@ public void setUp() startKafka(); pushAvroIntoKafka(_avroFiles); + setMaxSegmentCompletionTimeMillis(); // create schema for non-pauseless table Schema schema = createSchema(); schema.setSchemaName(DEFAULT_TABLE_NAME_2); @@ -206,24 +214,12 @@ private List getErroredSegmentsInExternalView(String tableName) { return erroredSegments; } - /** - * Checks that all segments which were previously in ERROR state now have status == UPLOADED. - */ - private boolean haveErroredSegmentsUploaded( - List segmentZKMetadataList, List previouslyErroredSegments) { - - // Convert to a Set for quick lookups - Set erroredSegmentNames = new HashSet<>(previouslyErroredSegments); - - for (SegmentZKMetadata metadata : segmentZKMetadataList) { - if (erroredSegmentNames.contains(metadata.getSegmentName())) { - // If it was previously ERROR, then we expect it to have transitioned to UPLOADED - if (metadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { - return false; - } - } + private void setMaxSegmentCompletionTimeMillis() { + PinotLLCRealtimeSegmentManager realtimeSegmentManager = _helixResourceManager.getRealtimeSegmentManager(); + if (realtimeSegmentManager instanceof FailureInjectingPinotLLCRealtimeSegmentManager) { + ((FailureInjectingPinotLLCRealtimeSegmentManager) realtimeSegmentManager) + .setMaxSegmentCompletionTimeoutMs(MAX_SEGMENT_COMPLETION_TIME_MILLIS); } - return true; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java index 66573e34b543..e737599d2c42 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingPinotLLCRealtimeSegmentManager.java @@ -26,11 +26,16 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; import org.apache.pinot.controller.helix.core.util.FailureInjectionUtils; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class FailureInjectingPinotLLCRealtimeSegmentManager extends PinotLLCRealtimeSegmentManager { @VisibleForTesting private final Map _failureConfig; + private long _maxSegmentCompletionTimeoutMs = 300000L; + private static final Logger LOGGER = LoggerFactory.getLogger(FailureInjectingPinotLLCRealtimeSegmentManager.class); public FailureInjectingPinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceManager, ControllerConf controllerConf, ControllerMetrics controllerMetrics) { @@ -38,6 +43,24 @@ public FailureInjectingPinotLLCRealtimeSegmentManager(PinotHelixResourceManager _failureConfig = new HashMap<>(); } + @Override + protected boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, + long currentTimeMs) { + Stat stat = new Stat(); + getSegmentZKMetadata(realtimeTableName, segmentName, stat); + if (currentTimeMs > stat.getMtime() + _maxSegmentCompletionTimeoutMs) { + LOGGER.info("Segment: {} exceeds the max completion time: {}ms, metadata update time: {}, current time: {}", + segmentName, _maxSegmentCompletionTimeoutMs, stat.getMtime(), currentTimeMs); + return true; + } else { + return false; + } + } + + public void setMaxSegmentCompletionTimeoutMs(long maxSegmentCompletionTimeoutMs) { + _maxSegmentCompletionTimeoutMs = maxSegmentCompletionTimeoutMs; + } + @VisibleForTesting public void enableTestFault(String faultType) { _failureConfig.put(faultType, "true"); From e84788a519a104fd2f942584ccbb06b250d3242d Mon Sep 17 00:00:00 2001 From: KKCorps Date: Wed, 29 Jan 2025 15:27:55 +0530 Subject: [PATCH 48/65] Fix error in tests post rebase --- .../realtime/FailureInjectingTableDataManagerProvider.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java index 0c323320d51a..983216e585ea 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java @@ -33,6 +33,7 @@ import org.apache.pinot.core.data.manager.provider.TableDataManagerProvider; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.utils.SegmentLocks; +import org.apache.pinot.segment.local.utils.SegmentPreprocessThrottler; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; @@ -51,7 +52,7 @@ public class FailureInjectingTableDataManagerProvider implements TableDataManage @Override public void init(InstanceDataManagerConfig instanceDataManagerConfig, HelixManager helixManager, - SegmentLocks segmentLocks) { + SegmentLocks segmentLocks, @Nullable SegmentPreprocessThrottler segmentPreprocessThrottler) { _instanceDataManagerConfig = instanceDataManagerConfig; _helixManager = helixManager; _segmentLocks = segmentLocks; @@ -87,7 +88,7 @@ public TableDataManager getTableDataManager(TableConfig tableConfig, @Nullable E throw new IllegalStateException(); } tableDataManager.init(_instanceDataManagerConfig, _helixManager, _segmentLocks, tableConfig, segmentPreloadExecutor, - errorCache); + errorCache, null); return tableDataManager; } } From 0d46327ab549897505f88141a71856b19aeddad4 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 30 Jan 2025 18:19:37 +0530 Subject: [PATCH 49/65] refactoring --- .../api/resources/ReIngestionResource.java | 33 +++++++++---------- ... StatelessRealtimeSegmentDataManager.java} | 6 ++-- 2 files changed, 19 insertions(+), 20 deletions(-) rename pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/{SimpleRealtimeSegmentDataManager.java => StatelessRealtimeSegmentDataManager.java} (98%) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 2d9eb10f5844..d0e1fe2ed732 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -34,6 +34,7 @@ import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -79,7 +80,7 @@ import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; -import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; +import org.apache.pinot.server.api.resources.reingestion.utils.StatelessRealtimeSegmentDataManager; import org.apache.pinot.server.realtime.ControllerLeaderLocator; import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.server.starter.ServerInstance; @@ -128,8 +129,9 @@ public class ReIngestionResource { // Keep track of jobs by jobId => job info private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); - public static final long CONSUMPTION_END_TIMEOUT_MS = 300000L; - public static final long UPLOAD_END_TIMEOUT_MS = 300000L; + public static final long CONSUMPTION_END_TIMEOUT_MS = Duration.ofMinutes(30).toMillis(); + public static final long UPLOAD_END_TIMEOUT_MS = Duration.ofMinutes(5).toMillis(); + public static final long CHECK_INTERVAL_MS = Duration.ofSeconds(5).toMillis(); @Inject private ServerInstance _serverInstance; @@ -250,12 +252,6 @@ public Response reIngestSegment(ReIngestionRequest request) { // Generate a jobId for tracking String jobId = UUID.randomUUID().toString(); ReIngestionJob job = new ReIngestionJob(jobId, tableNameWithType, segmentName); - RUNNING_JOBS.put(jobId, job); - - // Send immediate success response with jobId - ReIngestionResponse immediateResponse = new ReIngestionResponse( - "Re-ingestion job submitted successfully with jobId: " + jobId); - Response response = Response.ok(immediateResponse).build(); // Kick off the actual work asynchronously REINGESTION_EXECUTOR.submit(() -> { @@ -266,10 +262,11 @@ public Response reIngestSegment(ReIngestionRequest request) { Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - SimpleRealtimeSegmentDataManager manager = new SimpleRealtimeSegmentDataManager( + StatelessRealtimeSegmentDataManager manager = new StatelessRealtimeSegmentDataManager( segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, _serverInstance.getServerMetrics()); + RUNNING_JOBS.put(jobId, job); doReIngestSegment(manager, segmentName, tableNameWithType, indexLoadingConfig, tableDataManager); } catch (Exception e) { LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); @@ -279,20 +276,22 @@ public Response reIngestSegment(ReIngestionRequest request) { } }); - // Return immediately with the jobId - return response; + ReIngestionResponse immediateResponse = new ReIngestionResponse( + "Re-ingestion job submitted successfully with jobId: " + jobId); + return Response.ok(immediateResponse).build(); } /** * The actual re-ingestion logic, moved into a separate method for clarity. * This is essentially the old synchronous logic you had in reIngestSegment. */ - private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String segmentName, String tableNameWithType, - IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) + private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, String segmentName, + String tableNameWithType, IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) throws Exception { try { manager.startConsumption(); - waitForCondition((Void) -> manager.isDoneConsuming(), 1000, CONSUMPTION_END_TIMEOUT_MS, 0); + waitForCondition((Void) -> manager.isDoneConsuming(), CHECK_INTERVAL_MS, + CONSUMPTION_END_TIMEOUT_MS, 0); manager.stopConsumption(); if (!manager.isSuccess()) { @@ -300,7 +299,7 @@ private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String } LOGGER.info("Starting build for segment {}", segmentName); - SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + StatelessRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = manager.buildSegmentInternal(); File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); @@ -340,7 +339,7 @@ private void doReIngestSegment(SimpleRealtimeSegmentDataManager manager, String } SegmentDataManager segDataManager = tableDataManager.acquireSegment(segmentName); return segDataManager instanceof ImmutableSegmentDataManager; - }, 5000, UPLOAD_END_TIMEOUT_MS, 0); + }, CHECK_INTERVAL_MS, UPLOAD_END_TIMEOUT_MS, 0); // Trigger segment reset HttpClient httpClient = HttpClient.getInstance(); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java similarity index 98% rename from pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java rename to pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java index 34e17e276e2f..74cd5ec9c21a 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/SimpleRealtimeSegmentDataManager.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java @@ -79,7 +79,7 @@ /** * Simplified Segment Data Manager for ingesting data from a start offset to an end offset. */ -public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { +public class StatelessRealtimeSegmentDataManager extends SegmentDataManager { private static final int DEFAULT_CAPACITY = 100_000; private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; @@ -116,7 +116,7 @@ public class SimpleRealtimeSegmentDataManager extends SegmentDataManager { private volatile Throwable _consumptionException; private final ServerMetrics _serverMetrics; - public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, + public StatelessRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, ServerMetrics serverMetrics) @@ -133,7 +133,7 @@ public SimpleRealtimeSegmentDataManager(String segmentName, String tableNameWith _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis());; _serverMetrics = serverMetrics; - _logger = LoggerFactory.getLogger(SimpleRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); + _logger = LoggerFactory.getLogger(StatelessRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); _startOffset = _offsetFactory.create(startOffsetStr); From a94c7e3ad5e83a12af062c82730eb3e8aec6cbe6 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 3 Feb 2025 22:11:22 +0530 Subject: [PATCH 50/65] Remove redundant code --- .../pinot/common/utils/FileUploadDownloadClient.java | 10 +--------- .../realtime/PinotLLCRealtimeSegmentManager.java | 12 +++--------- .../validation/RealtimeSegmentValidationManager.java | 11 +++++------ 3 files changed, 9 insertions(+), 24 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 36d5570a29ff..cbeb1f656ab2 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -1305,15 +1305,7 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, requestJson.put("segmentName", segmentName); String jsonPayload = JsonUtils.objectToString(requestJson); - SimpleHttpResponse response = - HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); - - // Check that we got a 2xx response - int statusCode = response.getStatusCode(); - if (statusCode / 100 != 2) { - throw new IOException(String.format("Failed POST to %s, HTTP %d: %s", - reIngestUri, statusCode, response.getResponse())); - } + HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); } /** diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 3f7bec6f95f4..d6090656cb01 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2167,13 +2167,12 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { for (String segmentName : segmentsInErrorState) { // Skip non-LLC segments or segments missing from the ideal state altogether LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); - if (llcSegmentName == null || !segmentToInstanceCurrentStateMap.containsKey(segmentName)) { + if (llcSegmentName == null) { continue; } SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); - // We only consider segments that are in COMMITTING which is indicated by having an endOffset - // but have a missing or placeholder download URL + // We only consider segments that are in COMMITTING state if (segmentZKMetadata.getStatus() == Status.COMMITTING) { Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); @@ -2195,11 +2194,6 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { } catch (Exception e) { LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); } - } else if (segmentZKMetadata.getStatus() == Status.UPLOADED) { - LOGGER.info( - "Segment {} in table {} is in ERROR state with download URL present. Resetting segment to ONLINE state.", - segmentName, tableNameWithType); - _helixResourceManager.resetSegment(tableNameWithType, segmentName, null); } } } @@ -2225,7 +2219,7 @@ private String findAliveServerToReIngest(Set candidateServers) { } for (String server : candidateServers) { - if (liveInstances.contains(server)) { + if (liveInstances.contains(server) && instanceToEndpointMap.containsKey(server)) { return instanceToEndpointMap.get(server); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 2e26748e8ac2..c006a66e0c16 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -121,6 +121,11 @@ protected void processTable(String tableNameWithType, Context context) { } else { LOGGER.info("Skipping segment-level validation for table: {}", tableConfig.getTableName()); } + + boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); + if (isPauselessConsumptionEnabled) { + _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); + } } /** @@ -177,12 +182,6 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { // Update the total document count gauge _validationMetrics.updateTotalDocumentCountGauge(realtimeTableName, computeTotalDocumentCount(segmentsZKMetadata)); - boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); - - if (isPauselessConsumptionEnabled) { - _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); - } - // Check missing segments and upload them to the deep store if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); From 8b9b8d125f3580b5bd9c036a2302dd764e187b9c Mon Sep 17 00:00:00 2001 From: KKCorps Date: Mon, 3 Feb 2025 23:37:39 +0530 Subject: [PATCH 51/65] Add support for queue in reingestion --- .../api/resources/ReIngestionResource.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index d0e1fe2ed732..047d1bd85030 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -19,6 +19,7 @@ package org.apache.pinot.server.api.resources; import com.google.common.base.Function; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.swagger.annotations.Api; import io.swagger.annotations.ApiKeyAuthDefinition; import io.swagger.annotations.ApiOperation; @@ -42,7 +43,9 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -117,6 +120,7 @@ public class ReIngestionResource { private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); //TODO: Make this configurable + private static final int MIN_REINGESTION_THREADS = 4; private static final int MAX_PARALLEL_REINGESTIONS = 8; // Tracks if a particular segment is currently being re-ingested @@ -125,7 +129,9 @@ public class ReIngestionResource { // Executor for asynchronous re-ingestion private static final ExecutorService REINGESTION_EXECUTOR = - Executors.newFixedThreadPool(MAX_PARALLEL_REINGESTIONS); + new ThreadPoolExecutor(MIN_REINGESTION_THREADS, MAX_PARALLEL_REINGESTIONS, 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), // unbounded queue for the reingestion tasks + new ThreadFactoryBuilder().setNameFormat("reingestion-worker-%d").build()); // Keep track of jobs by jobId => job info private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); @@ -196,11 +202,6 @@ public Response reIngestSegment(ReIngestionRequest request) { String tableNameWithType = request.getTableNameWithType(); String segmentName = request.getSegmentName(); - if (RUNNING_JOBS.size() >= MAX_PARALLEL_REINGESTIONS) { - return Response.status(Response.Status.TOO_MANY_REQUESTS) - .entity("Reingestion jobs parallel limit " + MAX_PARALLEL_REINGESTIONS + " reached.").build(); - } - InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); if (instanceDataManager == null) { throw new WebApplicationException("Invalid server initialization", Response.Status.INTERNAL_SERVER_ERROR); @@ -233,6 +234,13 @@ public Response reIngestSegment(ReIngestionRequest request) { Response.Status.NOT_FOUND); } + // Check if download url is present + if (segmentZKMetadata.getDownloadUrl() != null) { + throw new WebApplicationException( + "Download URL is already present for segment: " + segmentName + ". No need to re-ingest.", + Response.Status.BAD_REQUEST); + } + // Grab start/end offsets String startOffsetStr = segmentZKMetadata.getStartOffset(); String endOffsetStr = segmentZKMetadata.getEndOffset(); From d035249200bdd1861019113f74fe72eb445771df Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 4 Feb 2025 20:15:38 +0530 Subject: [PATCH 52/65] Refactoring --- .../utils/FileUploadDownloadClient.java | 19 +- .../PinotLLCRealtimeSegmentManager.java | 71 +++---- .../realtime/RealtimeSegmentDataManager.java | 2 +- .../realtime/RealtimeTableDataManager.java | 1 + ...erverSegmentCompletionProtocolHandler.java | 158 +++++++++++++++ .../api/resources/ReIngestionResource.java | 190 ++---------------- 6 files changed, 213 insertions(+), 228 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index cbeb1f656ab2..b152faa57985 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -1279,13 +1279,9 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi * Invokes the server's reIngestSegment API via a POST request with JSON payload, * using Simple HTTP APIs. * - * POST http://[serverURL]/reIngestSegment - * { - * "tableNameWithType": [tableName], - * "segmentName": [segmentName] - * } + * POST http://[serverURL]/reIngestSegment/[segmentName] */ - public void triggerReIngestion(String serverHostPort, String tableNameWithType, String segmentName) + public void triggerReIngestion(String serverHostPort, String segmentName) throws IOException, URISyntaxException, HttpErrorStatusException { String scheme = HTTP; if (serverHostPort.contains(HTTPS)) { @@ -1298,14 +1294,9 @@ public void triggerReIngestion(String serverHostPort, String tableNameWithType, String serverHost = serverHostPort.split(":")[0]; String serverPort = serverHostPort.split(":")[1]; - URI reIngestUri = getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH); - - Map requestJson = new HashMap<>(); - requestJson.put("tableNameWithType", tableNameWithType); - requestJson.put("segmentName", segmentName); - - String jsonPayload = JsonUtils.objectToString(requestJson); - HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, jsonPayload)); + URI reIngestUri = + getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH + "/" + segmentName); + HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, "")); } /** diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index d6090656cb01..f47e653a8af3 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -49,10 +49,8 @@ import org.apache.helix.ClusterMessagingService; import org.apache.helix.Criteria; import org.apache.helix.HelixAdmin; -import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; -import org.apache.helix.PropertyKey; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; import org.apache.helix.store.zk.ZkHelixPropertyStore; @@ -446,10 +444,6 @@ public IdealState getIdealState(String realtimeTableName) { } } - public ExternalView getExternalView(String realtimeTableName) { - return _helixResourceManager.getTableExternalView(realtimeTableName); - } - @VisibleForTesting void setIdealState(String realtimeTableName, IdealState idealState) { try { @@ -2110,19 +2104,19 @@ URI createSegmentPath(String rawTableName, String segmentName) { * on one of the alive servers that are supposed to host that segment according to IdealState. * * API signature: - * POST http://[serverURL]/reIngestSegment + * POST http://[serverURL]/reIngestSegment/[segmentName] * Request body (JSON): - * { - * "tableNameWithType": [tableName], - * "segmentName": [segmentName] - * } * - * @param tableNameWithType The table name with type, e.g. "myTable_REALTIME" + * @param realtimeTableName The table name with type, e.g. "myTable_REALTIME" */ - public void reIngestSegmentsWithErrorState(String tableNameWithType) { + public void reIngestSegmentsWithErrorState(String realtimeTableName) { // Step 1: Fetch the ExternalView and all segments - ExternalView externalView = getExternalView(tableNameWithType); - IdealState idealState = getIdealState(tableNameWithType); + ExternalView externalView = _helixResourceManager.getTableExternalView(realtimeTableName); + if (externalView == null) { + LOGGER.warn("External view not found for table {}", realtimeTableName); + return; + } + IdealState idealState = getIdealState(realtimeTableName); Map> segmentToInstanceCurrentStateMap = externalView.getRecord().getMapFields(); Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); @@ -2130,6 +2124,13 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { List segmentsInErrorState = new ArrayList<>(); for (Map.Entry> entry : segmentToInstanceCurrentStateMap.entrySet()) { String segmentName = entry.getKey(); + + // Skip non-LLC segments or segments missing from the ideal state altogether + LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); + if (llcSegmentName == null) { + continue; + } + Map instanceStateMap = entry.getValue(); boolean allReplicasInError = true; for (String state : instanceStateMap.values()) { @@ -2144,7 +2145,7 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { } if (segmentsInErrorState.isEmpty()) { - LOGGER.info("No segments found in ERROR state for table {}", tableNameWithType); + LOGGER.info("No segments found in ERROR state for table {}", realtimeTableName); return; } @@ -2159,19 +2160,10 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { } } if (!isOnline) { - segmentsInErrorState.remove(segmentName); - } - } - - // Step 2: For each segment, check the ZK metadata for conditions - for (String segmentName : segmentsInErrorState) { - // Skip non-LLC segments or segments missing from the ideal state altogether - LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); - if (llcSegmentName == null) { continue; } - SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(tableNameWithType, segmentName); + SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(realtimeTableName, segmentName); // We only consider segments that are in COMMITTING state if (segmentZKMetadata.getStatus() == Status.COMMITTING) { Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); @@ -2179,17 +2171,17 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive LOGGER.info( "Segment {} in table {} is COMMITTING with missing download URL and no peer copy. Triggering re-ingestion.", - segmentName, tableNameWithType); + segmentName, realtimeTableName); // Find at least one server that should host this segment and is alive - String aliveServer = findAliveServerToReIngest(instanceStateMap.keySet()); + String aliveServer = pickServerToReIngest(instanceStateMap.keySet()); if (aliveServer == null) { - LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, tableNameWithType); + LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, realtimeTableName); continue; } try { - _fileUploadDownloadClient.triggerReIngestion(aliveServer, tableNameWithType, segmentName); + _fileUploadDownloadClient.triggerReIngestion(aliveServer, segmentName); LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); } catch (Exception e) { LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); @@ -2199,16 +2191,12 @@ public void reIngestSegmentsWithErrorState(String tableNameWithType) { } /** - * Picks one 'alive' server among a set of servers that are supposed to host the segment, - * e.g. by checking if Helix says it is enabled or if it appears in the live instance list. - * This is a simple example; adapt to your environment’s definition of “alive.” + * Picks one server among a set of servers that are supposed to host the segment, */ - private String findAliveServerToReIngest(Set candidateServers) { - // Get the current live instances from Helix - HelixDataAccessor helixDataAccessor = _helixManager.getHelixDataAccessor(); - PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder(); - List liveInstances = helixDataAccessor.getChildNames(keyBuilder.liveInstances()); + private String pickServerToReIngest(Set candidateServers) { try { + List serverList = new ArrayList<>(candidateServers); + String server = serverList.get(RANDOM.nextInt(serverList.size())); // This should ideally handle https scheme as well BiMap instanceToEndpointMap = _helixResourceManager.getDataInstanceAdminEndpoints(candidateServers); @@ -2218,11 +2206,8 @@ private String findAliveServerToReIngest(Set candidateServers) { return null; } - for (String server : candidateServers) { - if (liveInstances.contains(server) && instanceToEndpointMap.containsKey(server)) { - return instanceToEndpointMap.get(server); - } - } + // return random server + return instanceToEndpointMap.get(server); } catch (Exception e) { LOGGER.warn("Failed to get Helix instance data admin endpoints for servers: {}", candidateServers, e); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 5dffb866ac45..73a1c3b4f849 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -1243,7 +1243,7 @@ protected boolean buildSegmentAndReplace() return true; } - protected void closeStreamConsumers() { + private void closeStreamConsumers() { closePartitionGroupConsumer(); closePartitionMetadataProvider(); if (_acquiredConsumerSemaphore.compareAndSet(true, false)) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 4ed2778df939..73c8ca0adb05 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -637,6 +637,7 @@ private long getDownloadTimeOutMilliseconds(@Nullable TableConfig tableConfig) { }).orElse(DEFAULT_SEGMENT_DOWNLOAD_TIMEOUT_MS); } + @VisibleForTesting protected RealtimeSegmentDataManager createRealtimeSegmentDataManager(SegmentZKMetadata zkMetadata, TableConfig tableConfig, IndexLoadingConfig indexLoadingConfig, Schema schema, LLCSegmentName llcSegmentName, Semaphore semaphore, PartitionUpsertMetadataManager partitionUpsertMetadataManager, diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java index 593271df0c13..0a0ca0e061d1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java @@ -21,24 +21,46 @@ import java.io.File; import java.net.URI; import java.net.URISyntaxException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.UUID; import javax.net.ssl.SSLContext; +import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.NameValuePair; +import org.apache.hc.core5.http.message.BasicHeader; +import org.apache.hc.core5.http.message.BasicNameValuePair; import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.protocols.SegmentCompletionProtocol; import org.apache.pinot.common.utils.ClientSSLContextGenerator; import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.SimpleHttpResponse; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.common.utils.URIUtils; +import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.common.utils.http.HttpClientConfig; +import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.core.data.manager.realtime.Server2ControllerSegmentUploader; import org.apache.pinot.core.util.SegmentCompletionProtocolUtils; +import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.spi.auth.AuthProvider; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; import static org.apache.pinot.spi.utils.CommonConstants.Server.SegmentCompletionProtocol.*; @@ -258,4 +280,140 @@ private SegmentCompletionProtocol.Response sendCommitEndWithMetadataFiles(String SegmentCompletionProtocolUtils.raiseSegmentCompletionProtocolResponseMetric(_serverMetrics, response); return response; } + + public void uploadReingestedSegment(String segmentName, String segmentStoreUri, File segmentTarFile) + throws Exception { + List
headers = AuthProviderUtils.toRequestHeaders(_authProvider); + + LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); + String rawTableName = llcSegmentName.getTableName(); + String destUriStr = StringUtil.join(File.separator, segmentStoreUri, rawTableName, + SegmentCompletionUtils.generateTmpSegmentFileName(segmentName)); + + try (PinotFS pinotFS = PinotFSFactory.create(new URI(segmentStoreUri).getScheme())) { + URI destUri = new URI(destUriStr); + if (pinotFS.exists(destUri)) { + pinotFS.delete(destUri, true); + } + pinotFS.copyFromLocalFile(segmentTarFile, destUri); + } + + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); + String controllerUrl = getControllerUrl(rawTableName); + pushSegmentMetadata(llcSegmentName, controllerUrl, segmentTarFile, headers); + } + + /** + * Push segment metadata to the Pinot Controller in METADATA mode. + * + * @param llcSegmentName The LLC segment name + * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") + * @param segmentFile The local segment tar.gz file + * @param authHeaders A map of authentication or additional headers for the request + */ + public void pushSegmentMetadata(LLCSegmentName llcSegmentName, String controllerUrl, File segmentFile, + List
authHeaders) + throws Exception { + String segmentName = llcSegmentName.getSegmentName(); + String rawTableName = llcSegmentName.getTableName(); + + LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), + rawTableName, controllerUrl); + File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); + + LOGGER.info("Generated segment metadata tar file: {}", segmentMetadataFile.getAbsolutePath()); + try { + // Prepare headers + List
headers = authHeaders; + + // The upload type must be METADATA + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, + FileUploadDownloadClient.FileUploadType.METADATA.toString())); + + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); + + // Set table name parameter + List parameters = getSegmentPushCommonParams(rawTableName); + + // Construct the endpoint URI + URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); + + LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); + + // Perform the metadata upload + SimpleHttpResponse response = _fileUploadDownloadClient + .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, + HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + + LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, rawTableName, + controllerUrl, response.getStatusCode(), response.getResponse()); + } finally { + FileUtils.deleteQuietly(segmentMetadataFile); + } + } + + private List getSegmentPushCommonParams(String rawTableName) { + List params = new ArrayList<>(); + params.add( + new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION, "true")); + params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_NAME, rawTableName)); + params.add( + new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_TYPE, TableType.REALTIME.toString())); + return params; + } + + /** + * Generate a tar.gz file containing only the metadata files (metadata.properties, creation.meta) + * from a given Pinot segment tar.gz file. + */ + private File generateSegmentMetadataTar(File segmentTarFile) + throws Exception { + + if (!segmentTarFile.exists()) { + throw new IllegalArgumentException("Segment tar file does not exist: " + segmentTarFile.getAbsolutePath()); + } + + LOGGER.info("Generating segment metadata tar file from segment tar: {}", segmentTarFile.getAbsolutePath()); + File tempDir = Files.createTempDirectory("pinot-segment-temp").toFile(); + String uuid = UUID.randomUUID().toString(); + try { + File metadataDir = new File(tempDir, "segmentMetadataDir-" + uuid); + if (!metadataDir.mkdirs()) { + throw new RuntimeException("Failed to create metadata directory: " + metadataDir.getAbsolutePath()); + } + + LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, + new File(metadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); + + // Extract creation.meta + LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", segmentTarFile, metadataDir); + TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.SEGMENT_CREATION_META, + new File(metadataDir, V1Constants.SEGMENT_CREATION_META)); + + File segmentMetadataFile = + new File(FileUtils.getTempDirectory(), "segmentMetadata-" + UUID.randomUUID() + ".tar.gz"); + TarCompressionUtils.createCompressedTarFile(metadataDir, segmentMetadataFile); + return segmentMetadataFile; + } finally { + FileUtils.deleteQuietly(tempDir); + } + } + + public String getControllerUrl(String rawTableName) { + ControllerLeaderLocator leaderLocator = ControllerLeaderLocator.getInstance(); + final Pair leaderHostPort = leaderLocator.getControllerLeader(rawTableName); + if (leaderHostPort == null) { + LOGGER.warn("No leader found for table: {}", rawTableName); + return null; + } + Integer port = leaderHostPort.getRight(); + String protocol = _protocol; + if (_controllerHttpsPort != null) { + port = _controllerHttpsPort; + protocol = HTTPS_PROTOCOL; + } + + return URIUtils.buildURI(protocol, leaderHostPort.getLeft() + ":" + port, "", Collections.emptyMap()).toString(); + } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 047d1bd85030..94aaa24e2a3d 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -34,10 +34,8 @@ import java.net.URISyntaxException; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -54,45 +52,29 @@ import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hc.core5.http.Header; -import org.apache.hc.core5.http.NameValuePair; -import org.apache.hc.core5.http.message.BasicHeader; -import org.apache.hc.core5.http.message.BasicNameValuePair; import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.SimpleHttpResponse; -import org.apache.pinot.common.utils.TarCompressionUtils; -import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; -import org.apache.pinot.core.data.manager.realtime.SegmentCompletionUtils; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.segment.spi.V1Constants; -import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; import org.apache.pinot.server.api.resources.reingestion.utils.StatelessRealtimeSegmentDataManager; -import org.apache.pinot.server.realtime.ControllerLeaderLocator; import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.server.starter.ServerInstance; -import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.filesystem.PinotFS; -import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.IngestionConfigUtils; @@ -102,7 +84,6 @@ import org.slf4j.LoggerFactory; import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; -import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; @@ -189,7 +170,7 @@ public Response getAllRunningReingestionJobs() { } @POST - @Path("/reingestSegment") + @Path("/reingestSegment/{segmentName}") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) @ApiOperation(value = "Re-ingest segment asynchronously", notes = "Returns a jobId immediately; ingestion runs in " @@ -198,9 +179,14 @@ public Response getAllRunningReingestionJobs() { @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, message = "Internal server error", response = ErrorInfo.class) }) - public Response reIngestSegment(ReIngestionRequest request) { - String tableNameWithType = request.getTableNameWithType(); - String segmentName = request.getSegmentName(); + public Response reIngestSegment(@PathParam("segmentName") String segmentName) { + // if segment is not in LLC format, return error + if (!LLCSegmentName.isLLCSegment(segmentName)) { + throw new WebApplicationException("Segment name is not in LLC format: " + segmentName, + Response.Status.BAD_REQUEST); + } + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(llcSegmentName.getTableName()); InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); if (instanceDataManager == null) { @@ -264,7 +250,6 @@ public Response reIngestSegment(ReIngestionRequest request) { // Kick off the actual work asynchronously REINGESTION_EXECUTOR.submit(() -> { try { - LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); int partitionGroupId = llcSegmentName.getPartitionGroupId(); Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); @@ -275,7 +260,7 @@ public Response reIngestSegment(ReIngestionRequest request) { indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, _serverInstance.getServerMetrics()); RUNNING_JOBS.put(jobId, job); - doReIngestSegment(manager, segmentName, tableNameWithType, indexLoadingConfig, tableDataManager); + doReIngestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); } catch (Exception e) { LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); } finally { @@ -293,13 +278,14 @@ public Response reIngestSegment(ReIngestionRequest request) { * The actual re-ingestion logic, moved into a separate method for clarity. * This is essentially the old synchronous logic you had in reIngestSegment. */ - private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, String segmentName, + private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, LLCSegmentName llcSegmentName, String tableNameWithType, IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) throws Exception { try { + String segmentName = llcSegmentName.getSegmentName(); + manager.startConsumption(); - waitForCondition((Void) -> manager.isDoneConsuming(), CHECK_INTERVAL_MS, - CONSUMPTION_END_TIMEOUT_MS, 0); + waitForCondition((Void) -> manager.isDoneConsuming(), CHECK_INTERVAL_MS, CONSUMPTION_END_TIMEOUT_MS, 0); manager.stopConsumption(); if (!manager.isSuccess()) { @@ -318,26 +304,7 @@ private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, Stri ServerSegmentCompletionProtocolHandler protocolHandler = new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); - AuthProvider authProvider = protocolHandler.getAuthProvider(); - List
headers = AuthProviderUtils.toRequestHeaders(authProvider); - - String controllerUrl = getControllerUrl(tableNameWithType, protocolHandler); - - String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); - String segmentStoreUri = indexLoadingConfig.getSegmentStoreURI(); - String destUriStr = StringUtil.join(File.separator, segmentStoreUri, rawTableName, - SegmentCompletionUtils.generateTmpSegmentFileName(segmentName)); - - try (PinotFS pinotFS = PinotFSFactory.create(new URI(segmentStoreUri).getScheme())) { - URI destUri = new URI(destUriStr); - if (pinotFS.exists(destUri)) { - pinotFS.delete(destUri, true); - } - pinotFS.copyFromLocalFile(segmentTarFile, destUri); - } - - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); - pushSegmentMetadata(tableNameWithType, controllerUrl, segmentTarFile, headers, segmentName, protocolHandler); + protocolHandler.uploadReingestedSegment(segmentName, indexLoadingConfig.getSegmentStoreURI(), segmentTarFile); // Wait for segment to be uploaded waitForCondition((Void) -> { @@ -351,9 +318,10 @@ private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, Stri // Trigger segment reset HttpClient httpClient = HttpClient.getInstance(); - Map headersMap = headers.stream() - .collect(Collectors.toMap(Header::getName, Header::getValue)); - resetSegment(httpClient, controllerUrl, tableNameWithType, segmentName, null, headersMap); + List
headers = AuthProviderUtils.toRequestHeaders(protocolHandler.getAuthProvider()); + Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); + String controllerVipUrl = protocolHandler.getControllerUrl(llcSegmentName.getTableName()); + resetSegment(httpClient, controllerVipUrl, tableNameWithType, segmentName, null, headersMap); LOGGER.info("Re-ingested segment {} uploaded successfully", segmentName); } finally { @@ -412,122 +380,4 @@ private String getURLForSegmentReset(String controllerVipUrl, String tableNameWi private String encode(String s) { return URLEncoder.encode(s, StandardCharsets.UTF_8); } - - /** - * Push segment metadata to the Pinot Controller in METADATA mode. - * - * @param tableNameWithType The table name with type (e.g., "myTable_OFFLINE") - * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") - * @param segmentFile The local segment tar.gz file - * @param authHeaders A map of authentication or additional headers for the request - */ - public void pushSegmentMetadata(String tableNameWithType, String controllerUrl, File segmentFile, - List
authHeaders, String segmentName, ServerSegmentCompletionProtocolHandler protocolHandler) - throws Exception { - LOGGER.info("Pushing metadata of segment {} of table {} to controller: {}", segmentFile.getName(), - tableNameWithType, controllerUrl); - String tableName = tableNameWithType; - File segmentMetadataFile = generateSegmentMetadataTar(segmentFile); - - LOGGER.info("Generated segment metadata tar file: {}", segmentMetadataFile.getAbsolutePath()); - try { - // Prepare headers - List
headers = authHeaders; - - // The upload type must be METADATA - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, - FileUploadDownloadClient.FileUploadType.METADATA.toString())); - - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, "true")); - - // Set table name parameter - List parameters = getSegmentPushCommonParams(tableNameWithType); - - // Construct the endpoint URI - URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); - - LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); - - // Perform the metadata upload - SimpleHttpResponse response = protocolHandler.getFileUploadDownloadClient() - .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, - HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); - - LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, tableName, - controllerUrl, response.getStatusCode(), response.getResponse()); - } finally { - FileUtils.deleteQuietly(segmentMetadataFile); - } - } - - private List getSegmentPushCommonParams(String tableNameWithType) { - List params = new ArrayList<>(); - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION, - "true")); - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_NAME, - TableNameBuilder.extractRawTableName(tableNameWithType))); - TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); - if (tableType != null) { - params.add(new BasicNameValuePair(FileUploadDownloadClient.QueryParameters.TABLE_TYPE, tableType.toString())); - } else { - throw new RuntimeException(String.format("Failed to determine the tableType from name: %s", tableNameWithType)); - } - return params; - } - - /** - * Generate a tar.gz file containing only the metadata files (metadata.properties, creation.meta) - * from a given Pinot segment tar.gz file. - */ - private File generateSegmentMetadataTar(File segmentTarFile) - throws Exception { - - if (!segmentTarFile.exists()) { - throw new IllegalArgumentException("Segment tar file does not exist: " + segmentTarFile.getAbsolutePath()); - } - - LOGGER.info("Generating segment metadata tar file from segment tar: {}", segmentTarFile.getAbsolutePath()); - File tempDir = Files.createTempDirectory("pinot-segment-temp").toFile(); - String uuid = UUID.randomUUID().toString(); - try { - File metadataDir = new File(tempDir, "segmentMetadataDir-" + uuid); - if (!metadataDir.mkdirs()) { - throw new RuntimeException("Failed to create metadata directory: " + metadataDir.getAbsolutePath()); - } - - LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", segmentTarFile, metadataDir); - TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, - new File(metadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); - - // Extract creation.meta - LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", segmentTarFile, metadataDir); - TarCompressionUtils.untarOneFile(segmentTarFile, V1Constants.SEGMENT_CREATION_META, - new File(metadataDir, V1Constants.SEGMENT_CREATION_META)); - - File segmentMetadataFile = - new File(FileUtils.getTempDirectory(), "segmentMetadata-" + UUID.randomUUID() + ".tar.gz"); - TarCompressionUtils.createCompressedTarFile(metadataDir, segmentMetadataFile); - return segmentMetadataFile; - } finally { - FileUtils.deleteQuietly(tempDir); - } - } - - private String getControllerUrl(String rawTableName, ServerSegmentCompletionProtocolHandler protocolHandler) { - ControllerLeaderLocator leaderLocator = ControllerLeaderLocator.getInstance(); - final Pair leaderHostPort = leaderLocator.getControllerLeader(rawTableName); - if (leaderHostPort == null) { - LOGGER.warn("No leader found for table: {}", rawTableName); - return null; - } - Integer port = leaderHostPort.getRight(); - String protocol = protocolHandler.getProtocol(); - Integer controllerHttpsPort = protocolHandler.getControllerHttpsPort(); - if (controllerHttpsPort != null) { - port = controllerHttpsPort; - protocol = HTTPS_PROTOCOL; - } - - return URIUtils.buildURI(protocol, leaderHostPort.getLeft() + ":" + port, "", Collections.emptyMap()).toString(); - } } From bc8a65b8d66f8a21961b7ff5d2a3c2aba77313df Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 4 Feb 2025 21:20:05 +0530 Subject: [PATCH 53/65] refaactoring --- .../StatelessRealtimeSegmentWriter.java | 34 ++++--------------- .../api/resources/ReIngestionResource.java | 10 +++--- 2 files changed, 12 insertions(+), 32 deletions(-) rename pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java => pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java (95%) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java similarity index 95% rename from pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java rename to pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index 74cd5ec9c21a..5d3dffd53044 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.server.api.resources.reingestion.utils; +package org.apache.pinot.segment.local.realtime.writer; import com.google.common.annotations.VisibleForTesting; import java.io.File; @@ -33,9 +33,7 @@ import org.apache.commons.io.FileUtils; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metrics.ServerMetrics; -import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.TarCompressionUtils; -import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager; import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; @@ -44,7 +42,6 @@ import org.apache.pinot.segment.local.segment.creator.TransformPipeline; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.utils.IngestionUtils; -import org.apache.pinot.segment.spi.MutableSegment; import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; @@ -79,11 +76,10 @@ /** * Simplified Segment Data Manager for ingesting data from a start offset to an end offset. */ -public class StatelessRealtimeSegmentDataManager extends SegmentDataManager { +public class StatelessRealtimeSegmentWriter { private static final int DEFAULT_CAPACITY = 100_000; private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; - public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); private final String _segmentName; private final String _tableNameWithType; @@ -108,15 +104,13 @@ public class StatelessRealtimeSegmentDataManager extends SegmentDataManager { private final StreamPartitionMsgOffset _startOffset; private final StreamPartitionMsgOffset _endOffset; private volatile StreamPartitionMsgOffset _currentOffset; - private volatile int _numRowsIndexed = 0; - private final String _segmentStoreUriStr; private final int _fetchTimeoutMs; private final TransformPipeline _transformPipeline; private volatile boolean _isSuccess = false; private volatile Throwable _consumptionException; private final ServerMetrics _serverMetrics; - public StatelessRealtimeSegmentDataManager(String segmentName, String tableNameWithType, int partitionGroupId, + public StatelessRealtimeSegmentWriter(String segmentName, String tableNameWithType, int partitionGroupId, SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, ServerMetrics serverMetrics) @@ -128,12 +122,11 @@ public StatelessRealtimeSegmentDataManager(String segmentName, String tableNameW _segmentZKMetadata = segmentZKMetadata; _tableConfig = tableConfig; _schema = schema; - _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI(); _streamConfig = streamConfig; _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis());; _serverMetrics = serverMetrics; - _logger = LoggerFactory.getLogger(StatelessRealtimeSegmentDataManager.class.getName() + "_" + _segmentName); + _logger = LoggerFactory.getLogger(StatelessRealtimeSegmentWriter.class.getName() + "_" + _segmentName); _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); _startOffset = _offsetFactory.create(startOffsetStr); @@ -248,6 +241,7 @@ private class PartitionConsumer implements Runnable { public void run() { try { _consumer.start(_startOffset); + _logger.info("Created new consumer thread {} for {}", _consumerThread, this); _currentOffset = _startOffset; TransformPipeline.Result reusedResult = new TransformPipeline.Result(); while (!_shouldStop.get() && _currentOffset.compareTo(_endOffset) < 0) { @@ -278,10 +272,8 @@ public void run() { List transformedRows = reusedResult.getTransformedRows(); - // TODO: Do enrichment and transforms before indexing for (GenericRow transformedRow : transformedRows) { _realtimeSegment.index(transformedRow, streamMessage.getMetadata()); - _numRowsIndexed++; } } else { _logger.warn("Failed to decode message at offset {}: {}", _currentOffset, decodedResult.getException()); @@ -318,25 +310,13 @@ public void stopConsumption() { } } - @Override - public MutableSegment getSegment() { - return _realtimeSegment; - } - - @Override - public String getSegmentName() { - return _segmentName; - } - - @Override - protected void doDestroy() { + public void destroy() { _realtimeSegment.destroy(); FileUtils.deleteQuietly(_resourceTmpDir); FileUtils.deleteQuietly(_resourceDataDir); } - @Override - public void doOffload() { + public void offload() { stopConsumption(); } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 94aaa24e2a3d..589d22f6597a 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -68,9 +68,9 @@ import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.realtime.writer.StatelessRealtimeSegmentWriter; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; -import org.apache.pinot.server.api.resources.reingestion.utils.StatelessRealtimeSegmentDataManager; import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.server.starter.ServerInstance; import org.apache.pinot.spi.config.table.TableConfig; @@ -255,9 +255,9 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - StatelessRealtimeSegmentDataManager manager = new StatelessRealtimeSegmentDataManager( + StatelessRealtimeSegmentWriter manager = new StatelessRealtimeSegmentWriter( segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, tableConfig, schema, - indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, _serverInstance.getServerMetrics()); + indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, null); RUNNING_JOBS.put(jobId, job); doReIngestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); @@ -278,7 +278,7 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { * The actual re-ingestion logic, moved into a separate method for clarity. * This is essentially the old synchronous logic you had in reIngestSegment. */ - private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, LLCSegmentName llcSegmentName, + private void doReIngestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmentName llcSegmentName, String tableNameWithType, IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) throws Exception { try { @@ -293,7 +293,7 @@ private void doReIngestSegment(StatelessRealtimeSegmentDataManager manager, LLCS } LOGGER.info("Starting build for segment {}", segmentName); - StatelessRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + StatelessRealtimeSegmentWriter.SegmentBuildDescriptor segmentBuildDescriptor = manager.buildSegmentInternal(); File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); From f082d24f91f143a495bca8624c5da34fd02b2558 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Tue, 4 Feb 2025 21:42:10 +0530 Subject: [PATCH 54/65] Honour segment build semaphore during reingestion --- ...reInjectingRealtimeSegmentDataManager.java | 2 +- ...lureInjectingRealtimeTableDataManager.java | 2 +- ...lureInjectingTableDataManagerProvider.java | 2 +- .../StatelessRealtimeSegmentWriter.java | 31 +++++++++++++++++-- .../api/resources/ReIngestionResource.java | 11 ++++--- 5 files changed, 39 insertions(+), 9 deletions(-) rename pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/{ => utils}/FailureInjectingRealtimeSegmentDataManager.java (98%) rename pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/{ => utils}/FailureInjectingRealtimeTableDataManager.java (98%) rename pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/{ => utils}/FailureInjectingTableDataManagerProvider.java (98%) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeSegmentDataManager.java similarity index 98% rename from pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java rename to pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeSegmentDataManager.java index 072cc92e4923..960138e8a088 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeSegmentDataManager.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeSegmentDataManager.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.integration.tests.realtime; +package org.apache.pinot.integration.tests.realtime.utils; import java.util.concurrent.Semaphore; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeTableDataManager.java similarity index 98% rename from pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java rename to pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeTableDataManager.java index 7dfd703bd98a..2821b6373fd6 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingRealtimeTableDataManager.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingRealtimeTableDataManager.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.integration.tests.realtime; +package org.apache.pinot.integration.tests.realtime.utils; import java.util.concurrent.Semaphore; import java.util.function.BooleanSupplier; diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingTableDataManagerProvider.java similarity index 98% rename from pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java rename to pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingTableDataManagerProvider.java index 983216e585ea..7129f48e1a4b 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/FailureInjectingTableDataManagerProvider.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/realtime/utils/FailureInjectingTableDataManagerProvider.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.integration.tests.realtime; +package org.apache.pinot.integration.tests.realtime.utils; import com.google.common.cache.Cache; import java.util.Map; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index 5d3dffd53044..fd531e50cc72 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -22,11 +22,15 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.time.Duration; +import java.time.Instant; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; @@ -81,6 +85,7 @@ public class StatelessRealtimeSegmentWriter { private static final int DEFAULT_CAPACITY = 100_000; private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; + private final Semaphore _segBuildSemaphore; private final String _segmentName; private final String _tableNameWithType; private final int _partitionGroupId; @@ -113,9 +118,9 @@ public class StatelessRealtimeSegmentWriter { public StatelessRealtimeSegmentWriter(String segmentName, String tableNameWithType, int partitionGroupId, SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, - ServerMetrics serverMetrics) + Semaphore segBuildSemaphore, ServerMetrics serverMetrics) throws Exception { - + _segBuildSemaphore = segBuildSemaphore; _segmentName = segmentName; _tableNameWithType = tableNameWithType; _partitionGroupId = partitionGroupId; @@ -335,7 +340,27 @@ public Throwable getConsumptionException() { @VisibleForTesting public SegmentBuildDescriptor buildSegmentInternal() throws Exception { _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); + final long startTimeMillis = now(); + try { + if (_segBuildSemaphore != null) { + _logger.info("Trying to acquire semaphore for building segment"); + Instant acquireStart = Instant.now(); + int timeoutSeconds = 5; + while (!_segBuildSemaphore.tryAcquire(timeoutSeconds, TimeUnit.SECONDS)) { + _logger.warn("Could not acquire semaphore for building segment in {}", + Duration.between(acquireStart, Instant.now())); + timeoutSeconds = Math.min(timeoutSeconds * 2, 300); + } + _logger.info("Acquired semaphore for building segment"); + } + } catch (InterruptedException e) { + String errorMessage = "Interrupted while waiting for semaphore"; + _logger.error(errorMessage, e); + return null; + } + final long lockAcquireTimeMillis = now(); + final long waitTimeMillis = lockAcquireTimeMillis - startTimeMillis; // Build a segment from in-memory rows. // Use a temporary directory Path tempSegmentFolder = null; @@ -364,6 +389,8 @@ public SegmentBuildDescriptor buildSegmentInternal() throws Exception { return null; } final long buildTimeMillis = now() - lockAcquireTimeMillis; + _logger.info("Successfully built segment (Column Mode: {}) in {} ms, after lockWaitTime {} ms", + converter.isColumnMajorEnabled(), buildTimeMillis, waitTimeMillis); File dataDir = _resourceDataDir; File indexDir = new File(dataDir, _segmentNameStr); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index 589d22f6597a..a37c35ab283d 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -66,6 +66,7 @@ import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.realtime.writer.StatelessRealtimeSegmentWriter; @@ -193,7 +194,8 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { throw new WebApplicationException("Invalid server initialization", Response.Status.INTERNAL_SERVER_ERROR); } - TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); + RealtimeTableDataManager tableDataManager = + (RealtimeTableDataManager) instanceDataManager.getTableDataManager(tableNameWithType); if (tableDataManager == null) { throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, Response.Status.NOT_FOUND); @@ -255,9 +257,10 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - StatelessRealtimeSegmentWriter manager = new StatelessRealtimeSegmentWriter( - segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, tableConfig, schema, - indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, null); + StatelessRealtimeSegmentWriter manager = + new StatelessRealtimeSegmentWriter(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, + tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, + tableDataManager.getSegmentBuildSemaphore(), null); RUNNING_JOBS.put(jobId, job); doReIngestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); From d1ad30b31f37b775ea940f7d4c55e7d991f1c11c Mon Sep 17 00:00:00 2001 From: KKCorps Date: Wed, 5 Feb 2025 07:22:27 +0530 Subject: [PATCH 55/65] Fix test --- .../PauselessRealtimeIngestionSegmentCommitFailureTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java index b4489d4512cf..b924178bcbab 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PauselessRealtimeIngestionSegmentCommitFailureTest.java @@ -92,7 +92,7 @@ protected void overrideServerConf(PinotConfiguration serverConf) { serverConf.setProperty("pinot.server.instance." + HelixInstanceDataManagerConfig.UPLOAD_SEGMENT_TO_DEEP_STORE, "true"); serverConf.setProperty("pinot.server.instance." + CommonConstants.Server.TABLE_DATA_MANAGER_PROVIDER_CLASS, - "org.apache.pinot.integration.tests.realtime.FailureInjectingTableDataManagerProvider"); + "org.apache.pinot.integration.tests.realtime.utils.FailureInjectingTableDataManagerProvider"); } @Override From 7e47dd5f07d8835309c51b888d49034c44a42279 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Wed, 5 Feb 2025 15:52:59 +0530 Subject: [PATCH 56/65] Add a seperate API to upload reingested segments --- .../utils/FileUploadDownloadClient.java | 5 + ...tSegmentUploadDownloadRestletResource.java | 201 ++++++++++++++++++ .../controller/api/upload/ZKOperator.java | 93 ++++++++ .../data/manager/BaseTableDataManager.java | 19 +- ...erverSegmentCompletionProtocolHandler.java | 2 +- .../api/resources/ReIngestionResource.java | 25 --- 6 files changed, 308 insertions(+), 37 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index b152faa57985..e6055a218a85 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -370,6 +370,11 @@ public static URI getUploadSegmentURI(URI controllerURI) return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), SEGMENT_PATH); } + public static URI getReingestSegmentURI(URI controllerURI) + throws URISyntaxException { + return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), "/segments/reingest"); + } + public static URI getBatchSegmentUploadURI(URI controllerURI) throws URISyntaxException { return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index cf8c2bf956f3..91cb703e3152 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -566,6 +566,166 @@ private SuccessResponse uploadSegments(String tableName, TableType tableType, Fo segmentNames, tableNameWithType, System.currentTimeMillis() - segmentsUploadStartTimeMs)); } + private SuccessResponse reingestRealtimeSegment(@Nullable String tableName, TableType tableType, + @Nullable FormDataMultiPart multiPart, boolean enableParallelPushProtection, HttpHeaders headers, + Request request) { + if (StringUtils.isNotEmpty(tableName)) { + TableType tableTypeFromTableName = TableNameBuilder.getTableTypeFromTableName(tableName); + if (tableType != TableType.REALTIME) { + throw new ControllerApplicationException(LOGGER, + String.format("Only REALTIME table type is supported for reingestion, but got table type: %s", tableType), + Response.Status.BAD_REQUEST); + } + if (tableTypeFromTableName != null && tableTypeFromTableName != tableType) { + throw new ControllerApplicationException(LOGGER, + String.format("Table name: %s does not match table type: %s", tableName, tableType), + Response.Status.BAD_REQUEST); + } + } + + // TODO: Consider validating the segment name and table name from the header against the actual segment + extractHttpHeader(headers, CommonConstants.Controller.SEGMENT_NAME_HTTP_HEADER); + extractHttpHeader(headers, CommonConstants.Controller.TABLE_NAME_HTTP_HEADER); + + String sourceDownloadURIStr = extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI); + String crypterClassNameInHeader = extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.CRYPTER); + String ingestionDescriptor = extractHttpHeader(headers, CommonConstants.Controller.INGESTION_DESCRIPTOR); + + File tempEncryptedFile = null; + File tempDecryptedFile = null; + File tempSegmentDir = null; + // The downloadUri for putting into segment zk metadata + String segmentDownloadURIStr = sourceDownloadURIStr; + try { + ControllerFilePathProvider provider = ControllerFilePathProvider.getInstance(); + String tempFileName = TMP_DIR_PREFIX + UUID.randomUUID(); + tempEncryptedFile = new File(provider.getFileUploadTempDir(), tempFileName + ENCRYPTED_SUFFIX); + tempDecryptedFile = new File(provider.getFileUploadTempDir(), tempFileName); + tempSegmentDir = new File(provider.getUntarredFileTempDir(), tempFileName); + + boolean uploadedSegmentIsEncrypted = StringUtils.isNotEmpty(crypterClassNameInHeader); + File destFile = uploadedSegmentIsEncrypted ? tempEncryptedFile : tempDecryptedFile; + long segmentSizeInBytes; + + // only support METADATA upload type for reingestion + if (multiPart == null) { + throw new ControllerApplicationException(LOGGER, + "Segment metadata file (as multipart/form-data) is required for METADATA upload mode", + Response.Status.BAD_REQUEST); + } + if (StringUtils.isEmpty(sourceDownloadURIStr)) { + throw new ControllerApplicationException(LOGGER, + "Source download URI is required in header field 'DOWNLOAD_URI' for METADATA upload mode", + Response.Status.BAD_REQUEST); + } + // override copySegmentToFinalLocation if override provided in headers:COPY_SEGMENT_TO_DEEP_STORE + // else set to false for backward compatibility + String copySegmentToDeepStore = + extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE); + boolean copySegmentToFinalLocation = Boolean.parseBoolean(copySegmentToDeepStore); + createSegmentFileFromMultipart(multiPart, destFile); + PinotFS pinotFS = null; + try { + URI segmentURI = new URI(sourceDownloadURIStr); + pinotFS = PinotFSFactory.create(segmentURI.getScheme()); + segmentSizeInBytes = pinotFS.length(segmentURI); + } catch (Exception e) { + segmentSizeInBytes = -1; + LOGGER.warn("Could not fetch segment size for metadata push", e); + } finally { + if (pinotFS != null) { + pinotFS.close(); + } + } + + if (uploadedSegmentIsEncrypted) { + decryptFile(crypterClassNameInHeader, tempEncryptedFile, tempDecryptedFile); + } + + String metadataProviderClass = DefaultMetadataExtractor.class.getName(); + SegmentMetadata segmentMetadata = getSegmentMetadata(tempDecryptedFile, tempSegmentDir, metadataProviderClass); + + // Fetch segment name + String segmentName = segmentMetadata.getName(); + + // Fetch table name. Try to derive the table name from the parameter and then from segment metadata + String rawTableName; + if (StringUtils.isNotEmpty(tableName)) { + rawTableName = TableNameBuilder.extractRawTableName(tableName); + } else { + // TODO: remove this when we completely deprecate the table name from segment metadata + rawTableName = segmentMetadata.getTableName(); + LOGGER.warn("Table name is not provided as request query parameter when uploading segment: {} for table: {}", + segmentName, rawTableName); + } + String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); + + String clientAddress = InetAddress.getByName(request.getRemoteAddr()).getHostName(); + LOGGER.info("Processing reingest request for segment: {} of table: {} from client: {}, " + + "ingestion descriptor: {}", segmentName, tableNameWithType, clientAddress, ingestionDescriptor); + + // Validate segment + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + if (tableConfig == null) { + throw new ControllerApplicationException(LOGGER, "Failed to find table: " + tableNameWithType, + Response.Status.BAD_REQUEST); + } + if (tableConfig.getIngestionConfig() == null || tableConfig.getIngestionConfig().isSegmentTimeValueCheck()) { + SegmentValidationUtils.validateTimeInterval(segmentMetadata, tableConfig); + } + long untarredSegmentSizeInBytes; + // TODO: Include the untarred segment size when using the METADATA push rest API. Currently we can only use the + // tarred segment size as an approximation. + untarredSegmentSizeInBytes = segmentSizeInBytes; + SegmentValidationUtils.checkStorageQuota(segmentName, untarredSegmentSizeInBytes, tableConfig, + _storageQuotaChecker); + + // Encrypt segment + String crypterNameInTableConfig = tableConfig.getValidationConfig().getCrypterClassName(); + Pair encryptionInfo = + encryptSegmentIfNeeded(tempDecryptedFile, tempEncryptedFile, uploadedSegmentIsEncrypted, + crypterClassNameInHeader, crypterNameInTableConfig, segmentName, tableNameWithType); + + String crypterName = encryptionInfo.getLeft(); + File segmentFile = encryptionInfo.getRight(); + + // Update download URI if controller is responsible for moving the segment to the deep store + URI finalSegmentLocationURI = null; + if (copySegmentToFinalLocation) { + URI dataDirURI = provider.getDataDirURI(); + String dataDirPath = dataDirURI.toString(); + String encodedSegmentName = URIUtils.encode(segmentName); + String finalSegmentLocationPath = URIUtils.getPath(dataDirPath, rawTableName, encodedSegmentName); + if (dataDirURI.getScheme().equalsIgnoreCase(CommonConstants.Segment.LOCAL_SEGMENT_SCHEME)) { + segmentDownloadURIStr = URIUtils.getPath(provider.getVip(), "segments", rawTableName, encodedSegmentName); + } else { + segmentDownloadURIStr = finalSegmentLocationPath; + } + finalSegmentLocationURI = URIUtils.getUri(finalSegmentLocationPath); + } + LOGGER.info("Using segment download URI: {} for segment: {} of table: {} (move segment: {})", + segmentDownloadURIStr, segmentFile, tableNameWithType, copySegmentToFinalLocation); + + ZKOperator zkOperator = new ZKOperator(_pinotHelixResourceManager, _controllerConf, _controllerMetrics); + zkOperator.updateReingestedSegmentZKMetadata(tableNameWithType, segmentMetadata, finalSegmentLocationURI, + sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, + headers); + + return new SuccessResponse("Successfully uploaded segment: " + segmentName + " of table: " + tableNameWithType); + } catch (WebApplicationException e) { + throw e; + } catch (Exception e) { + _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_SEGMENT_UPLOAD_ERROR, 1L); + _controllerMetrics.addMeteredTableValue(tableName, ControllerMeter.CONTROLLER_TABLE_SEGMENT_UPLOAD_ERROR, 1L); + throw new ControllerApplicationException(LOGGER, "Exception while uploading segment: " + e.getMessage(), + Response.Status.INTERNAL_SERVER_ERROR, e); + } finally { + FileUtils.deleteQuietly(tempEncryptedFile); + FileUtils.deleteQuietly(tempDecryptedFile); + FileUtils.deleteQuietly(tempSegmentDir); + } + } + private void cleanupTempFiles(List tempFiles) { for (File tempFile : tempFiles) { FileUtils.deleteQuietly(tempFile); @@ -965,6 +1125,47 @@ public Response revertReplaceSegments( } } + @POST + @ManagedAsync + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.MULTIPART_FORM_DATA) + @Path("segments/reingest") + @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.UPLOAD_SEGMENT) + @Authenticate(AccessType.CREATE) + @ApiOperation(value = "Reingest a realtime segment", notes = "Reingest a segment as multipart file") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Successfully reingested segment"), + @ApiResponse(code = 400, message = "Bad Request"), + @ApiResponse(code = 403, message = "Segment validation fails"), + @ApiResponse(code = 409, message = "Segment already exists or another parallel push in progress"), + @ApiResponse(code = 410, message = "Segment to refresh does not exist"), + @ApiResponse(code = 412, message = "CRC check fails"), + @ApiResponse(code = 500, message = "Internal error") + }) + @TrackInflightRequestMetrics + @TrackedByGauge(gauge = ControllerGauge.SEGMENT_UPLOADS_IN_PROGRESS) + // We use this endpoint with URI upload because a request sent with the multipart content type will reject the POST + // request if a multipart object is not sent. This endpoint is recommended for use. It differs from the first + // endpoint in how it moves the segment to a Pinot-determined final directory. + public void reingestSegment(FormDataMultiPart multiPart, + @ApiParam(value = "Name of the table") @QueryParam(FileUploadDownloadClient.QueryParameters.TABLE_NAME) + String tableName, + @ApiParam(value = "Type of the table") @QueryParam(FileUploadDownloadClient.QueryParameters.TABLE_TYPE) + @DefaultValue("OFFLINE") String tableType, + @ApiParam(value = "Whether to enable parallel push protection") @DefaultValue("false") + @QueryParam(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION) + boolean enableParallelPushProtection, + @ApiParam(value = "Whether to refresh if the segment already exists") @DefaultValue("true") + @QueryParam(FileUploadDownloadClient.QueryParameters.ALLOW_REFRESH) boolean allowRefresh, + @Context HttpHeaders headers, @Context Request request, @Suspended final AsyncResponse asyncResponse) { + try { + asyncResponse.resume(reingestRealtimeSegment(tableName, TableType.valueOf(tableType.toUpperCase()), multiPart, + enableParallelPushProtection, headers, request)); + } catch (Throwable t) { + asyncResponse.resume(t); + } + } + private static void createSegmentFileFromMultipart(FormDataMultiPart multiPart, File destFile) throws IOException { // Read segment file or segment metadata file and directly use that information to update zk diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index 4da9c72a5665..39a1775aa5a8 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -170,6 +170,26 @@ public void completeSegmentsOperations(String tableNameWithType, FileUploadType processExistingSegments(tableNameWithType, uploadType, enableParallelPushProtection, headers, existingSegmentsList); } + public void updateReingestedSegmentZKMetadata(String tableNameWithType, SegmentMetadata segmentMetadata, + @Nullable URI finalSegmentLocationURI, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, + @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) + throws Exception { + String segmentName = segmentMetadata.getName(); + ZNRecord existingSegmentMetadataZNRecord = + _pinotHelixResourceManager.getSegmentMetadataZnRecord(tableNameWithType, segmentName); + + if (existingSegmentMetadataZNRecord == null) { + throw new ControllerApplicationException(LOGGER, + String.format("Segment: %s does not exist in table: %s", segmentName, tableNameWithType), + Response.Status.NOT_FOUND); + } + + // Refresh an existing segment + processReIngestedSegment(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, + finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, + segmentSizeInBytes, enableParallelPushProtection, headers); + } + /** * Returns {@code true} when the segment should be processed as new segment. *

When segment ZK metadata exists, check if segment exists in the ideal state. If the previous upload failed after @@ -474,6 +494,79 @@ private void processExistingSegments(String tableNameWithType, FileUploadType up } } + private void processReIngestedSegment(String tableNameWithType, SegmentMetadata segmentMetadata, + ZNRecord existingSegmentMetadataZNRecord, @Nullable URI finalSegmentLocationURI, + @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, + long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) + throws Exception { + String segmentName = segmentMetadata.getName(); + int expectedVersion = existingSegmentMetadataZNRecord.getVersion(); + + // Check if CRC match when IF-MATCH header is set + SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(existingSegmentMetadataZNRecord); + long existingCrc = segmentZKMetadata.getCrc(); + checkCRC(headers, tableNameWithType, segmentName, existingCrc); + + // Check segment upload start time when parallel push protection enabled + if (enableParallelPushProtection) { + // When segment upload start time is larger than 0, that means another upload is in progress + long segmentUploadStartTime = segmentZKMetadata.getSegmentUploadStartTime(); + if (segmentUploadStartTime > 0) { + handleParallelPush(tableNameWithType, segmentName, segmentUploadStartTime); + } + + // Lock the segment by setting the upload start time in ZK + segmentZKMetadata.setSegmentUploadStartTime(System.currentTimeMillis()); + if (!_pinotHelixResourceManager.updateZkMetadata(tableNameWithType, segmentZKMetadata, expectedVersion)) { + throw new ControllerApplicationException(LOGGER, + String.format("Failed to lock the segment: %s of table: %s, retry later", segmentName, tableNameWithType), + Response.Status.CONFLICT); + } else { + // The version will increment if the zk metadata update is successful + expectedVersion++; + } + } + + // Reset segment upload start time to unlock the segment later + // NOTE: reset this value even if parallel push protection is not enabled so that segment can recover in case + // previous segment upload did not finish properly and the parallel push protection is turned off + segmentZKMetadata.setSegmentUploadStartTime(-1); + + try { + // Update ZK metadata and refresh the segment if necessary + long newCrc = Long.parseLong(segmentMetadata.getCrc()); + // New segment is different with the existing one, update ZK metadata and refresh the segment + LOGGER.info( + "New segment crc {} is different than the existing segment crc {}. Updating ZK metadata and refreshing " + + "segment {}", newCrc, existingCrc, segmentName); + if (finalSegmentLocationURI != null) { + copyFromSegmentURIToDeepStore(new URI(sourceDownloadURIStr), finalSegmentLocationURI); + LOGGER.info("Copied segment: {} of table: {} to final location: {}", segmentName, tableNameWithType, + finalSegmentLocationURI); + } + + // If no modifier is provided, use the custom map from the segment metadata + segmentZKMetadata.setCustomMap(segmentMetadata.getCustomMap()); + SegmentZKMetadataUtils.updateCommittingSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, + segmentDownloadURIStr, segmentSizeInBytes, segmentZKMetadata.getEndOffset()); + if (!_pinotHelixResourceManager.updateZkMetadata(tableNameWithType, segmentZKMetadata, expectedVersion)) { + throw new RuntimeException( + String.format("Failed to update ZK metadata for segment: %s, table: %s, expected version: %d", + segmentName, tableNameWithType, expectedVersion)); + } + LOGGER.info("Updated segment: {} of table: {} to property store", segmentName, tableNameWithType); + + // Send a message to servers hosting the table to reset the segment + _pinotHelixResourceManager.resetSegment(tableNameWithType, segmentName, null); + } catch (Exception e) { + if (!_pinotHelixResourceManager.updateZkMetadata(tableNameWithType, segmentZKMetadata, expectedVersion)) { + LOGGER.error("Failed to update ZK metadata for segment: {}, table: {}, expected version: {}", segmentName, + tableNameWithType, expectedVersion); + } + throw e; + } + } + private void checkCRC(HttpHeaders headers, String tableNameWithType, String segmentName, long existingCrc) { String expectedCrcStr = headers.getHeaderString(HttpHeaders.IF_MATCH); if (expectedCrcStr != null) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index d69034270987..23af663545b4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -56,7 +56,6 @@ import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; -import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.common.utils.config.TierConfigUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; @@ -394,17 +393,15 @@ protected void replaceSegmentIfCrcMismatch(SegmentDataManager segmentDataManager // For pauseless tables, we should replace the segment if download url is missing even if crc is same // Without this the reingestion of ERROR segments in pauseless tables fails // as the segment data manager is still an instance of RealtimeSegmentDataManager - if (!PauselessConsumptionUtils.isPauselessEnabled(tableConfig)) { - Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, - "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); - SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); - if (hasSameCRC(zkMetadata, localMetadata)) { - _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); - return; - } - _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, - localMetadata.getCrc(), zkMetadata.getCrc()); + Preconditions.checkState(segmentDataManager instanceof ImmutableSegmentDataManager, + "Cannot replace CONSUMING segment: %s in table: %s", segmentName, _tableNameWithType); + SegmentMetadata localMetadata = segmentDataManager.getSegment().getSegmentMetadata(); + if (hasSameCRC(zkMetadata, localMetadata)) { + _logger.info("Segment: {} has CRC: {} same as before, not replacing it", segmentName, localMetadata.getCrc()); + return; } + _logger.info("Replacing segment: {} because its CRC has changed from: {} to: {}", segmentName, + localMetadata.getCrc(), zkMetadata.getCrc()); downloadAndLoadSegment(zkMetadata, indexLoadingConfig); _logger.info("Replaced segment: {} with new CRC: {}", segmentName, zkMetadata.getCrc()); } diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java index 0a0ca0e061d1..553c9614a729 100644 --- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java @@ -336,7 +336,7 @@ public void pushSegmentMetadata(LLCSegmentName llcSegmentName, String controller List parameters = getSegmentPushCommonParams(rawTableName); // Construct the endpoint URI - URI uploadEndpoint = FileUploadDownloadClient.getUploadSegmentURI(new URI(controllerUrl)); + URI uploadEndpoint = FileUploadDownloadClient.getReingestSegmentURI(new URI(controllerUrl)); LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java index a37c35ab283d..797b8857649f 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java @@ -45,7 +45,6 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.inject.Inject; import javax.ws.rs.Consumes; @@ -58,16 +57,12 @@ import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import org.apache.hc.core5.http.Header; -import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; -import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; -import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.realtime.writer.StatelessRealtimeSegmentWriter; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; @@ -77,7 +72,6 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -306,26 +300,7 @@ private void doReIngestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmen ServerSegmentCompletionProtocolHandler protocolHandler = new ServerSegmentCompletionProtocolHandler(_serverInstance.getServerMetrics(), tableNameWithType); - protocolHandler.uploadReingestedSegment(segmentName, indexLoadingConfig.getSegmentStoreURI(), segmentTarFile); - - // Wait for segment to be uploaded - waitForCondition((Void) -> { - SegmentZKMetadata zkMetadata = tableDataManager.fetchZKMetadata(segmentName); - if (zkMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.UPLOADED) { - return false; - } - SegmentDataManager segDataManager = tableDataManager.acquireSegment(segmentName); - return segDataManager instanceof ImmutableSegmentDataManager; - }, CHECK_INTERVAL_MS, UPLOAD_END_TIMEOUT_MS, 0); - - // Trigger segment reset - HttpClient httpClient = HttpClient.getInstance(); - List

headers = AuthProviderUtils.toRequestHeaders(protocolHandler.getAuthProvider()); - Map headersMap = headers.stream().collect(Collectors.toMap(Header::getName, Header::getValue)); - String controllerVipUrl = protocolHandler.getControllerUrl(llcSegmentName.getTableName()); - resetSegment(httpClient, controllerVipUrl, tableNameWithType, segmentName, null, headersMap); - LOGGER.info("Re-ingested segment {} uploaded successfully", segmentName); } finally { manager.offload(); From 5a42c28e351d72ac1043bed3f97997f85c8df163 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Wed, 5 Feb 2025 18:47:44 +0530 Subject: [PATCH 57/65] Cleanup code --- .../pinot/common/metrics/ControllerGauge.java | 5 +- .../utils/FileUploadDownloadClient.java | 31 +-- ...tSegmentUploadDownloadRestletResource.java | 187 ++---------------- .../controller/api/upload/ZKOperator.java | 40 +++- .../PinotLLCRealtimeSegmentManager.java | 31 ++- ...erverSegmentCompletionProtocolHandler.java | 11 +- 6 files changed, 87 insertions(+), 218 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java index 99bd89206607..dfa1f1e09d95 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java @@ -180,7 +180,10 @@ public enum ControllerGauge implements AbstractMetrics.Gauge { // segment when the partition is first detected). COMMITTING_SEGMENT_SIZE("committingSegmentSize", false), - TABLE_REBALANCE_IN_PROGRESS("tableRebalanceInProgress", false); + TABLE_REBALANCE_IN_PROGRESS("tableRebalanceInProgress", false), + + // Number of in progress segment reingestion + SEGMENT_REINGESTION_IN_PROGRESS("segmentReingestionInProgress", true); private final String _gaugeName; private final String _unit; diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index e6055a218a85..542ed2e058c3 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -127,7 +127,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; - public static final String REINGEST_SEGMENT_PATH = "/reingestSegment"; + public static final String SEGMENT_REINGEST_COMPLETION_PATH = "/segment/completeReingestion"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); @@ -370,9 +370,10 @@ public static URI getUploadSegmentURI(URI controllerURI) return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), SEGMENT_PATH); } - public static URI getReingestSegmentURI(URI controllerURI) + public static URI getSegmentReingestCompletionURI(URI controllerURI) throws URISyntaxException { - return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), "/segments/reingest"); + return getURI(controllerURI.getScheme(), controllerURI.getHost(), controllerURI.getPort(), + SEGMENT_REINGEST_COMPLETION_PATH); } public static URI getBatchSegmentUploadURI(URI controllerURI) @@ -1280,30 +1281,6 @@ public File downloadUntarFileStreamed(URI uri, File dest, AuthProvider authProvi httpHeaders, maxStreamRateInByte); } - /** - * Invokes the server's reIngestSegment API via a POST request with JSON payload, - * using Simple HTTP APIs. - * - * POST http://[serverURL]/reIngestSegment/[segmentName] - */ - public void triggerReIngestion(String serverHostPort, String segmentName) - throws IOException, URISyntaxException, HttpErrorStatusException { - String scheme = HTTP; - if (serverHostPort.contains(HTTPS)) { - scheme = HTTPS; - serverHostPort = serverHostPort.replace(HTTPS + "://", ""); - } else if (serverHostPort.contains(HTTP)) { - serverHostPort = serverHostPort.replace(HTTP + "://", ""); - } - - String serverHost = serverHostPort.split(":")[0]; - String serverPort = serverHostPort.split(":")[1]; - - URI reIngestUri = - getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH + "/" + segmentName); - HttpClient.wrapAndThrowHttpException(_httpClient.sendJsonPostRequest(reIngestUri, "")); - } - /** * Generate a param list with a table name attribute. * diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index 91cb703e3152..3aa3f304c46d 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -233,6 +233,13 @@ public Response downloadSegment( private SuccessResponse uploadSegment(@Nullable String tableName, TableType tableType, @Nullable FormDataMultiPart multiPart, boolean copySegmentToFinalLocation, boolean enableParallelPushProtection, boolean allowRefresh, HttpHeaders headers, Request request) { + return uploadSegment(tableName, tableType, multiPart, copySegmentToFinalLocation, enableParallelPushProtection, + allowRefresh, false, headers, request); + } + + private SuccessResponse uploadSegment(@Nullable String tableName, TableType tableType, + @Nullable FormDataMultiPart multiPart, boolean copySegmentToFinalLocation, boolean enableParallelPushProtection, + boolean allowRefresh, boolean allowReset, HttpHeaders headers, Request request) { if (StringUtils.isNotEmpty(tableName)) { TableType tableTypeFromTableName = TableNameBuilder.getTableTypeFromTableName(tableName); if (tableTypeFromTableName != null && tableTypeFromTableName != tableType) { @@ -411,7 +418,7 @@ private SuccessResponse uploadSegment(@Nullable String tableName, TableType tabl ZKOperator zkOperator = new ZKOperator(_pinotHelixResourceManager, _controllerConf, _controllerMetrics); zkOperator.completeSegmentOperations(tableNameWithType, segmentMetadata, uploadType, finalSegmentLocationURI, segmentFile, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, - enableParallelPushProtection, allowRefresh, headers); + enableParallelPushProtection, allowRefresh, allowReset, headers); return new SuccessResponse("Successfully uploaded segment: " + segmentName + " of table: " + tableNameWithType); } catch (WebApplicationException e) { @@ -566,166 +573,6 @@ private SuccessResponse uploadSegments(String tableName, TableType tableType, Fo segmentNames, tableNameWithType, System.currentTimeMillis() - segmentsUploadStartTimeMs)); } - private SuccessResponse reingestRealtimeSegment(@Nullable String tableName, TableType tableType, - @Nullable FormDataMultiPart multiPart, boolean enableParallelPushProtection, HttpHeaders headers, - Request request) { - if (StringUtils.isNotEmpty(tableName)) { - TableType tableTypeFromTableName = TableNameBuilder.getTableTypeFromTableName(tableName); - if (tableType != TableType.REALTIME) { - throw new ControllerApplicationException(LOGGER, - String.format("Only REALTIME table type is supported for reingestion, but got table type: %s", tableType), - Response.Status.BAD_REQUEST); - } - if (tableTypeFromTableName != null && tableTypeFromTableName != tableType) { - throw new ControllerApplicationException(LOGGER, - String.format("Table name: %s does not match table type: %s", tableName, tableType), - Response.Status.BAD_REQUEST); - } - } - - // TODO: Consider validating the segment name and table name from the header against the actual segment - extractHttpHeader(headers, CommonConstants.Controller.SEGMENT_NAME_HTTP_HEADER); - extractHttpHeader(headers, CommonConstants.Controller.TABLE_NAME_HTTP_HEADER); - - String sourceDownloadURIStr = extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI); - String crypterClassNameInHeader = extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.CRYPTER); - String ingestionDescriptor = extractHttpHeader(headers, CommonConstants.Controller.INGESTION_DESCRIPTOR); - - File tempEncryptedFile = null; - File tempDecryptedFile = null; - File tempSegmentDir = null; - // The downloadUri for putting into segment zk metadata - String segmentDownloadURIStr = sourceDownloadURIStr; - try { - ControllerFilePathProvider provider = ControllerFilePathProvider.getInstance(); - String tempFileName = TMP_DIR_PREFIX + UUID.randomUUID(); - tempEncryptedFile = new File(provider.getFileUploadTempDir(), tempFileName + ENCRYPTED_SUFFIX); - tempDecryptedFile = new File(provider.getFileUploadTempDir(), tempFileName); - tempSegmentDir = new File(provider.getUntarredFileTempDir(), tempFileName); - - boolean uploadedSegmentIsEncrypted = StringUtils.isNotEmpty(crypterClassNameInHeader); - File destFile = uploadedSegmentIsEncrypted ? tempEncryptedFile : tempDecryptedFile; - long segmentSizeInBytes; - - // only support METADATA upload type for reingestion - if (multiPart == null) { - throw new ControllerApplicationException(LOGGER, - "Segment metadata file (as multipart/form-data) is required for METADATA upload mode", - Response.Status.BAD_REQUEST); - } - if (StringUtils.isEmpty(sourceDownloadURIStr)) { - throw new ControllerApplicationException(LOGGER, - "Source download URI is required in header field 'DOWNLOAD_URI' for METADATA upload mode", - Response.Status.BAD_REQUEST); - } - // override copySegmentToFinalLocation if override provided in headers:COPY_SEGMENT_TO_DEEP_STORE - // else set to false for backward compatibility - String copySegmentToDeepStore = - extractHttpHeader(headers, FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE); - boolean copySegmentToFinalLocation = Boolean.parseBoolean(copySegmentToDeepStore); - createSegmentFileFromMultipart(multiPart, destFile); - PinotFS pinotFS = null; - try { - URI segmentURI = new URI(sourceDownloadURIStr); - pinotFS = PinotFSFactory.create(segmentURI.getScheme()); - segmentSizeInBytes = pinotFS.length(segmentURI); - } catch (Exception e) { - segmentSizeInBytes = -1; - LOGGER.warn("Could not fetch segment size for metadata push", e); - } finally { - if (pinotFS != null) { - pinotFS.close(); - } - } - - if (uploadedSegmentIsEncrypted) { - decryptFile(crypterClassNameInHeader, tempEncryptedFile, tempDecryptedFile); - } - - String metadataProviderClass = DefaultMetadataExtractor.class.getName(); - SegmentMetadata segmentMetadata = getSegmentMetadata(tempDecryptedFile, tempSegmentDir, metadataProviderClass); - - // Fetch segment name - String segmentName = segmentMetadata.getName(); - - // Fetch table name. Try to derive the table name from the parameter and then from segment metadata - String rawTableName; - if (StringUtils.isNotEmpty(tableName)) { - rawTableName = TableNameBuilder.extractRawTableName(tableName); - } else { - // TODO: remove this when we completely deprecate the table name from segment metadata - rawTableName = segmentMetadata.getTableName(); - LOGGER.warn("Table name is not provided as request query parameter when uploading segment: {} for table: {}", - segmentName, rawTableName); - } - String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); - - String clientAddress = InetAddress.getByName(request.getRemoteAddr()).getHostName(); - LOGGER.info("Processing reingest request for segment: {} of table: {} from client: {}, " - + "ingestion descriptor: {}", segmentName, tableNameWithType, clientAddress, ingestionDescriptor); - - // Validate segment - TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); - if (tableConfig == null) { - throw new ControllerApplicationException(LOGGER, "Failed to find table: " + tableNameWithType, - Response.Status.BAD_REQUEST); - } - if (tableConfig.getIngestionConfig() == null || tableConfig.getIngestionConfig().isSegmentTimeValueCheck()) { - SegmentValidationUtils.validateTimeInterval(segmentMetadata, tableConfig); - } - long untarredSegmentSizeInBytes; - // TODO: Include the untarred segment size when using the METADATA push rest API. Currently we can only use the - // tarred segment size as an approximation. - untarredSegmentSizeInBytes = segmentSizeInBytes; - SegmentValidationUtils.checkStorageQuota(segmentName, untarredSegmentSizeInBytes, tableConfig, - _storageQuotaChecker); - - // Encrypt segment - String crypterNameInTableConfig = tableConfig.getValidationConfig().getCrypterClassName(); - Pair encryptionInfo = - encryptSegmentIfNeeded(tempDecryptedFile, tempEncryptedFile, uploadedSegmentIsEncrypted, - crypterClassNameInHeader, crypterNameInTableConfig, segmentName, tableNameWithType); - - String crypterName = encryptionInfo.getLeft(); - File segmentFile = encryptionInfo.getRight(); - - // Update download URI if controller is responsible for moving the segment to the deep store - URI finalSegmentLocationURI = null; - if (copySegmentToFinalLocation) { - URI dataDirURI = provider.getDataDirURI(); - String dataDirPath = dataDirURI.toString(); - String encodedSegmentName = URIUtils.encode(segmentName); - String finalSegmentLocationPath = URIUtils.getPath(dataDirPath, rawTableName, encodedSegmentName); - if (dataDirURI.getScheme().equalsIgnoreCase(CommonConstants.Segment.LOCAL_SEGMENT_SCHEME)) { - segmentDownloadURIStr = URIUtils.getPath(provider.getVip(), "segments", rawTableName, encodedSegmentName); - } else { - segmentDownloadURIStr = finalSegmentLocationPath; - } - finalSegmentLocationURI = URIUtils.getUri(finalSegmentLocationPath); - } - LOGGER.info("Using segment download URI: {} for segment: {} of table: {} (move segment: {})", - segmentDownloadURIStr, segmentFile, tableNameWithType, copySegmentToFinalLocation); - - ZKOperator zkOperator = new ZKOperator(_pinotHelixResourceManager, _controllerConf, _controllerMetrics); - zkOperator.updateReingestedSegmentZKMetadata(tableNameWithType, segmentMetadata, finalSegmentLocationURI, - sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, - headers); - - return new SuccessResponse("Successfully uploaded segment: " + segmentName + " of table: " + tableNameWithType); - } catch (WebApplicationException e) { - throw e; - } catch (Exception e) { - _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_SEGMENT_UPLOAD_ERROR, 1L); - _controllerMetrics.addMeteredTableValue(tableName, ControllerMeter.CONTROLLER_TABLE_SEGMENT_UPLOAD_ERROR, 1L); - throw new ControllerApplicationException(LOGGER, "Exception while uploading segment: " + e.getMessage(), - Response.Status.INTERNAL_SERVER_ERROR, e); - } finally { - FileUtils.deleteQuietly(tempEncryptedFile); - FileUtils.deleteQuietly(tempDecryptedFile); - FileUtils.deleteQuietly(tempSegmentDir); - } - } - private void cleanupTempFiles(List tempFiles) { for (File tempFile : tempFiles) { FileUtils.deleteQuietly(tempFile); @@ -1129,7 +976,7 @@ public Response revertReplaceSegments( @ManagedAsync @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.MULTIPART_FORM_DATA) - @Path("segments/reingest") + @Path("segment/completeReingestion") @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.UPLOAD_SEGMENT) @Authenticate(AccessType.CREATE) @ApiOperation(value = "Reingest a realtime segment", notes = "Reingest a segment as multipart file") @@ -1138,29 +985,23 @@ public Response revertReplaceSegments( @ApiResponse(code = 400, message = "Bad Request"), @ApiResponse(code = 403, message = "Segment validation fails"), @ApiResponse(code = 409, message = "Segment already exists or another parallel push in progress"), - @ApiResponse(code = 410, message = "Segment to refresh does not exist"), @ApiResponse(code = 412, message = "CRC check fails"), @ApiResponse(code = 500, message = "Internal error") }) @TrackInflightRequestMetrics - @TrackedByGauge(gauge = ControllerGauge.SEGMENT_UPLOADS_IN_PROGRESS) - // We use this endpoint with URI upload because a request sent with the multipart content type will reject the POST - // request if a multipart object is not sent. This endpoint is recommended for use. It differs from the first - // endpoint in how it moves the segment to a Pinot-determined final directory. - public void reingestSegment(FormDataMultiPart multiPart, + @TrackedByGauge(gauge = ControllerGauge.SEGMENT_REINGESTION_IN_PROGRESS) + public void completeSegmentReingestion(FormDataMultiPart multiPart, @ApiParam(value = "Name of the table") @QueryParam(FileUploadDownloadClient.QueryParameters.TABLE_NAME) String tableName, @ApiParam(value = "Type of the table") @QueryParam(FileUploadDownloadClient.QueryParameters.TABLE_TYPE) - @DefaultValue("OFFLINE") String tableType, + @DefaultValue("REALTIME") String tableType, @ApiParam(value = "Whether to enable parallel push protection") @DefaultValue("false") @QueryParam(FileUploadDownloadClient.QueryParameters.ENABLE_PARALLEL_PUSH_PROTECTION) boolean enableParallelPushProtection, - @ApiParam(value = "Whether to refresh if the segment already exists") @DefaultValue("true") - @QueryParam(FileUploadDownloadClient.QueryParameters.ALLOW_REFRESH) boolean allowRefresh, @Context HttpHeaders headers, @Context Request request, @Suspended final AsyncResponse asyncResponse) { try { - asyncResponse.resume(reingestRealtimeSegment(tableName, TableType.valueOf(tableType.toUpperCase()), multiPart, - enableParallelPushProtection, headers, request)); + asyncResponse.resume(uploadSegment(tableName, TableType.valueOf(tableType.toUpperCase()), multiPart, + true, enableParallelPushProtection, false, true, headers, request)); } catch (Throwable t) { asyncResponse.resume(t); } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index 39a1775aa5a8..920543701f52 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -72,6 +72,17 @@ public void completeSegmentOperations(String tableNameWithType, SegmentMetadata @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, boolean allowRefresh, HttpHeaders headers) throws Exception { + completeSegmentOperations(tableNameWithType, segmentMetadata, uploadType, finalSegmentLocationURI, segmentFile, + sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, + allowRefresh, false, headers); + } + + public void completeSegmentOperations(String tableNameWithType, SegmentMetadata segmentMetadata, + FileUploadType uploadType, @Nullable URI finalSegmentLocationURI, File segmentFile, + @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, + long segmentSizeInBytes, boolean enableParallelPushProtection, boolean allowRefresh, boolean allowReset, + HttpHeaders headers) + throws Exception { String segmentName = segmentMetadata.getName(); boolean refreshOnly = Boolean.parseBoolean(headers.getHeaderString(FileUploadDownloadClient.CustomHeaders.REFRESH_ONLY)); @@ -100,18 +111,27 @@ public void completeSegmentOperations(String tableNameWithType, SegmentMetadata headers); } else { // Refresh an existing segment - if (!allowRefresh) { + + if (!(allowRefresh || allowReset)) { // We cannot perform this check up-front in UploadSegment API call. If a segment doesn't exist during the check // done up-front but ends up getting created before the check here, we could incorrectly refresh an existing // segment. throw new ControllerApplicationException(LOGGER, - String.format("Segment: %s already exists in table: %s. Refresh not permitted.", segmentName, - tableNameWithType), Response.Status.CONFLICT); + String.format("Segment: %s already exists in table: %s. Neither refresh nor reset not permitted.", + segmentName, tableNameWithType), Response.Status.CONFLICT); + } + + if (allowRefresh) { + LOGGER.info("Segment: {} already exists in table: {}, refreshing it", segmentName, tableNameWithType); + processExistingSegmentWithRefresh(tableNameWithType, segmentMetadata, uploadType, + existingSegmentMetadataZNRecord, finalSegmentLocationURI, segmentFile, sourceDownloadURIStr, + segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, headers); + } else if (allowReset) { + LOGGER.info("Segment: {} already exists in table: {}, resetting it", segmentName, tableNameWithType); + processExistingSegmentWithReset(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, + finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, + enableParallelPushProtection, headers); } - LOGGER.info("Segment: {} already exists in table: {}, refreshing it", segmentName, tableNameWithType); - processExistingSegment(tableNameWithType, segmentMetadata, uploadType, existingSegmentMetadataZNRecord, - finalSegmentLocationURI, segmentFile, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, - segmentSizeInBytes, enableParallelPushProtection, headers); } } @@ -185,7 +205,7 @@ public void updateReingestedSegmentZKMetadata(String tableNameWithType, SegmentM } // Refresh an existing segment - processReIngestedSegment(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, + processExistingSegmentWithReset(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, headers); } @@ -228,7 +248,7 @@ private void handleParallelPush(String tableNameWithType, String segmentName, lo } } - private void processExistingSegment(String tableNameWithType, SegmentMetadata segmentMetadata, + private void processExistingSegmentWithRefresh(String tableNameWithType, SegmentMetadata segmentMetadata, FileUploadType uploadType, ZNRecord existingSegmentMetadataZNRecord, @Nullable URI finalSegmentLocationURI, File segmentFile, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) @@ -494,7 +514,7 @@ private void processExistingSegments(String tableNameWithType, FileUploadType up } } - private void processReIngestedSegment(String tableNameWithType, SegmentMetadata segmentMetadata, + private void processExistingSegmentWithReset(String tableNameWithType, SegmentMetadata segmentMetadata, ZNRecord existingSegmentMetadataZNRecord, @Nullable URI finalSegmentLocationURI, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index f47e653a8af3..f5c74fe2010f 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -26,6 +26,7 @@ import com.google.common.collect.Sets; import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Arrays; @@ -57,6 +58,7 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.assignment.InstancePartitions; import org.apache.pinot.common.assignment.InstancePartitionsUtils; +import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.messages.ForceCommitMessage; import org.apache.pinot.common.messages.IngestionMetricsRemoveMessage; import org.apache.pinot.common.metadata.ZKMetadataProvider; @@ -73,6 +75,7 @@ import org.apache.pinot.common.utils.PauselessConsumptionUtils; import org.apache.pinot.common.utils.URIUtils; import org.apache.pinot.common.utils.helix.HelixHelper; +import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.controller.api.events.MetadataEventNotifierFactory; import org.apache.pinot.controller.api.resources.Constants; @@ -170,6 +173,8 @@ public class PinotLLCRealtimeSegmentManager { * deep store fix if necessary. RetentionManager will delete this kind of segments shortly anyway. */ private static final long MIN_TIME_BEFORE_SEGMENT_EXPIRATION_FOR_FIXING_DEEP_STORE_COPY_MILLIS = 60 * 60 * 1000L; + private static final String REINGEST_SEGMENT_PATH = "/reingestSegment"; + // 1 hour private static final Random RANDOM = new Random(); @@ -2181,7 +2186,7 @@ public void reIngestSegmentsWithErrorState(String realtimeTableName) { } try { - _fileUploadDownloadClient.triggerReIngestion(aliveServer, segmentName); + triggerReIngestion(aliveServer, segmentName); LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); } catch (Exception e) { LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); @@ -2190,6 +2195,30 @@ public void reIngestSegmentsWithErrorState(String realtimeTableName) { } } + /** + * Invokes the server's reIngestSegment API via a POST request with JSON payload, + * using Simple HTTP APIs. + * + * POST http://[serverURL]/reIngestSegment/[segmentName] + */ + private void triggerReIngestion(String serverHostPort, String segmentName) + throws IOException, URISyntaxException, HttpErrorStatusException { + String scheme = CommonConstants.HTTP_PROTOCOL; + if (serverHostPort.contains(CommonConstants.HTTPS_PROTOCOL)) { + scheme = CommonConstants.HTTPS_PROTOCOL; + serverHostPort = serverHostPort.replace(CommonConstants.HTTPS_PROTOCOL + "://", ""); + } else if (serverHostPort.contains(CommonConstants.HTTP_PROTOCOL)) { + serverHostPort = serverHostPort.replace(CommonConstants.HTTP_PROTOCOL + "://", ""); + } + + String serverHost = serverHostPort.split(":")[0]; + String serverPort = serverHostPort.split(":")[1]; + + URI reIngestUri = FileUploadDownloadClient.getURI(scheme, serverHost, Integer.parseInt(serverPort), + REINGEST_SEGMENT_PATH + "/" + segmentName); + HttpClient.wrapAndThrowHttpException(HttpClient.getInstance().sendJsonPostRequest(reIngestUri, "")); + } + /** * Picks one server among a set of servers that are supposed to host the segment, */ diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java index 553c9614a729..fe47778fdcf0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java @@ -60,7 +60,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.pinot.spi.utils.CommonConstants.HTTPS_PROTOCOL; import static org.apache.pinot.spi.utils.CommonConstants.Server.SegmentCompletionProtocol.*; @@ -300,18 +299,18 @@ public void uploadReingestedSegment(String segmentName, String segmentStoreUri, headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, destUriStr)); String controllerUrl = getControllerUrl(rawTableName); - pushSegmentMetadata(llcSegmentName, controllerUrl, segmentTarFile, headers); + triggerSegmentReingestCompletion(llcSegmentName, controllerUrl, segmentTarFile, headers); } /** - * Push segment metadata to the Pinot Controller in METADATA mode. + * Trigger the segment reingest completion protocol to the controller. * * @param llcSegmentName The LLC segment name * @param controllerUrl The base URL of the Pinot Controller (e.g., "http://controller-host:9000") * @param segmentFile The local segment tar.gz file * @param authHeaders A map of authentication or additional headers for the request */ - public void pushSegmentMetadata(LLCSegmentName llcSegmentName, String controllerUrl, File segmentFile, + public void triggerSegmentReingestCompletion(LLCSegmentName llcSegmentName, String controllerUrl, File segmentFile, List
authHeaders) throws Exception { String segmentName = llcSegmentName.getSegmentName(); @@ -335,8 +334,8 @@ public void pushSegmentMetadata(LLCSegmentName llcSegmentName, String controller // Set table name parameter List parameters = getSegmentPushCommonParams(rawTableName); - // Construct the endpoint URI - URI uploadEndpoint = FileUploadDownloadClient.getReingestSegmentURI(new URI(controllerUrl)); + // + URI uploadEndpoint = FileUploadDownloadClient.getSegmentReingestCompletionURI(new URI(controllerUrl)); LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); From 155168579b263f0c5627b07b38d977f0b9d7a479 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 08:57:54 +0530 Subject: [PATCH 58/65] Replace reIngest with reingest --- .../utils/FileUploadDownloadClient.java | 2 +- ...tSegmentUploadDownloadRestletResource.java | 2 +- .../controller/api/upload/ZKOperator.java | 20 ------------ .../PinotLLCRealtimeSegmentManager.java | 26 ++++++++-------- .../RealtimeSegmentValidationManager.java | 2 +- ...erverSegmentCompletionProtocolHandler.java | 7 ++--- ...Resource.java => ReingestionResource.java} | 31 +++++++++---------- ...onRequest.java => ReingestionRequest.java} | 2 +- ...Response.java => ReingestionResponse.java} | 4 +-- 9 files changed, 37 insertions(+), 59 deletions(-) rename pinot-server/src/main/java/org/apache/pinot/server/api/resources/{ReIngestionResource.java => ReingestionResource.java} (94%) rename pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/{ReIngestionRequest.java => ReingestionRequest.java} (97%) rename pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/{ReIngestionResponse.java => ReingestionResponse.java} (92%) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 542ed2e058c3..162af381225e 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -127,7 +127,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; - public static final String SEGMENT_REINGEST_COMPLETION_PATH = "/segment/completeReingestion"; + public static final String SEGMENT_REINGEST_COMPLETION_PATH = "/segment/reingested"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index 3aa3f304c46d..92125555ba3b 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -976,7 +976,7 @@ public Response revertReplaceSegments( @ManagedAsync @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.MULTIPART_FORM_DATA) - @Path("segment/completeReingestion") + @Path("segment/reingested") @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.UPLOAD_SEGMENT) @Authenticate(AccessType.CREATE) @ApiOperation(value = "Reingest a realtime segment", notes = "Reingest a segment as multipart file") diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index 920543701f52..a10dd8075502 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -190,26 +190,6 @@ public void completeSegmentsOperations(String tableNameWithType, FileUploadType processExistingSegments(tableNameWithType, uploadType, enableParallelPushProtection, headers, existingSegmentsList); } - public void updateReingestedSegmentZKMetadata(String tableNameWithType, SegmentMetadata segmentMetadata, - @Nullable URI finalSegmentLocationURI, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, - @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) - throws Exception { - String segmentName = segmentMetadata.getName(); - ZNRecord existingSegmentMetadataZNRecord = - _pinotHelixResourceManager.getSegmentMetadataZnRecord(tableNameWithType, segmentName); - - if (existingSegmentMetadataZNRecord == null) { - throw new ControllerApplicationException(LOGGER, - String.format("Segment: %s does not exist in table: %s", segmentName, tableNameWithType), - Response.Status.NOT_FOUND); - } - - // Refresh an existing segment - processExistingSegmentWithReset(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, - finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, - segmentSizeInBytes, enableParallelPushProtection, headers); - } - /** * Returns {@code true} when the segment should be processed as new segment. *

When segment ZK metadata exists, check if segment exists in the ideal state. If the previous upload failed after diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index f5c74fe2010f..9f6c2298b141 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2105,16 +2105,16 @@ URI createSegmentPath(String rawTableName, String segmentName) { /** * Re-ingests segments that are in ERROR state in EV but ONLINE in IS with no peer copy on any server. This method - * will call the server reIngestSegment API + * will call the server reingestSegment API * on one of the alive servers that are supposed to host that segment according to IdealState. * * API signature: - * POST http://[serverURL]/reIngestSegment/[segmentName] + * POST http://[serverURL]/reingestSegment/[segmentName] * Request body (JSON): * * @param realtimeTableName The table name with type, e.g. "myTable_REALTIME" */ - public void reIngestSegmentsWithErrorState(String realtimeTableName) { + public void reingestSegmentsWithErrorState(String realtimeTableName) { // Step 1: Fetch the ExternalView and all segments ExternalView externalView = _helixResourceManager.getTableExternalView(realtimeTableName); if (externalView == null) { @@ -2179,29 +2179,29 @@ public void reIngestSegmentsWithErrorState(String realtimeTableName) { segmentName, realtimeTableName); // Find at least one server that should host this segment and is alive - String aliveServer = pickServerToReIngest(instanceStateMap.keySet()); + String aliveServer = pickServerToReingest(instanceStateMap.keySet()); if (aliveServer == null) { LOGGER.warn("No alive server found to re-ingest segment {} in table {}", segmentName, realtimeTableName); continue; } try { - triggerReIngestion(aliveServer, segmentName); - LOGGER.info("Successfully triggered reIngestion for segment {} on server {}", segmentName, aliveServer); + triggerReingestion(aliveServer, segmentName); + LOGGER.info("Successfully triggered reingestion for segment {} on server {}", segmentName, aliveServer); } catch (Exception e) { - LOGGER.error("Failed to call reIngestSegment for segment {} on server {}", segmentName, aliveServer, e); + LOGGER.error("Failed to call reingestSegment for segment {} on server {}", segmentName, aliveServer, e); } } } } /** - * Invokes the server's reIngestSegment API via a POST request with JSON payload, + * Invokes the server's reingestSegment API via a POST request with JSON payload, * using Simple HTTP APIs. * - * POST http://[serverURL]/reIngestSegment/[segmentName] + * POST http://[serverURL]/reingestSegment/[segmentName] */ - private void triggerReIngestion(String serverHostPort, String segmentName) + private void triggerReingestion(String serverHostPort, String segmentName) throws IOException, URISyntaxException, HttpErrorStatusException { String scheme = CommonConstants.HTTP_PROTOCOL; if (serverHostPort.contains(CommonConstants.HTTPS_PROTOCOL)) { @@ -2214,15 +2214,15 @@ private void triggerReIngestion(String serverHostPort, String segmentName) String serverHost = serverHostPort.split(":")[0]; String serverPort = serverHostPort.split(":")[1]; - URI reIngestUri = FileUploadDownloadClient.getURI(scheme, serverHost, Integer.parseInt(serverPort), + URI reingestUri = FileUploadDownloadClient.getURI(scheme, serverHost, Integer.parseInt(serverPort), REINGEST_SEGMENT_PATH + "/" + segmentName); - HttpClient.wrapAndThrowHttpException(HttpClient.getInstance().sendJsonPostRequest(reIngestUri, "")); + HttpClient.wrapAndThrowHttpException(HttpClient.getInstance().sendJsonPostRequest(reingestUri, "")); } /** * Picks one server among a set of servers that are supposed to host the segment, */ - private String pickServerToReIngest(Set candidateServers) { + private String pickServerToReingest(Set candidateServers) { try { List serverList = new ArrayList<>(candidateServers); String server = serverList.get(RANDOM.nextInt(serverList.size())); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index c006a66e0c16..09212c037022 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -124,7 +124,7 @@ protected void processTable(String tableNameWithType, Context context) { boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); if (isPauselessConsumptionEnabled) { - _llcRealtimeSegmentManager.reIngestSegmentsWithErrorState(tableConfig.getTableName()); + _llcRealtimeSegmentManager.reingestSegmentsWithErrorState(tableConfig.getTableName()); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java index fe47778fdcf0..b94c9b5bcca4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java @@ -334,14 +334,13 @@ public void triggerSegmentReingestCompletion(LLCSegmentName llcSegmentName, Stri // Set table name parameter List parameters = getSegmentPushCommonParams(rawTableName); - // - URI uploadEndpoint = FileUploadDownloadClient.getSegmentReingestCompletionURI(new URI(controllerUrl)); + URI reingestCompletionURI = FileUploadDownloadClient.getSegmentReingestCompletionURI(new URI(controllerUrl)); - LOGGER.info("Uploading segment metadata to: {} with headers: {}", uploadEndpoint, headers); + LOGGER.info("Uploading segment metadata to: {} with headers: {}", reingestCompletionURI, headers); // Perform the metadata upload SimpleHttpResponse response = _fileUploadDownloadClient - .uploadSegmentMetadata(uploadEndpoint, segmentName, segmentMetadataFile, headers, parameters, + .uploadSegmentMetadata(reingestCompletionURI, segmentName, segmentMetadataFile, headers, parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); LOGGER.info("Response for pushing metadata of segment {} of table {} to {} - {}: {}", segmentName, rawTableName, diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java similarity index 94% rename from pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java rename to pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java index 797b8857649f..c6e80c50666b 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java @@ -66,7 +66,7 @@ import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.realtime.writer.StatelessRealtimeSegmentWriter; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; +import org.apache.pinot.server.api.resources.reingestion.ReingestionResponse; import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; import org.apache.pinot.server.starter.ServerInstance; import org.apache.pinot.spi.config.table.TableConfig; @@ -82,7 +82,7 @@ import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; -@Api(tags = "ReIngestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), +@Api(tags = "Reingestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), @Authorization(value = DATABASE)}) @SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = { @ApiKeyAuthDefinition(name = HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, @@ -92,8 +92,8 @@ description = "Database context passed through http header. If no context is provided 'default' database " + "context will be considered.")})) @Path("/") -public class ReIngestionResource { - private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); +public class ReingestionResource { + private static final Logger LOGGER = LoggerFactory.getLogger(ReingestionResource.class); //TODO: Make this configurable private static final int MIN_REINGESTION_THREADS = 4; @@ -110,9 +110,8 @@ public class ReIngestionResource { new ThreadFactoryBuilder().setNameFormat("reingestion-worker-%d").build()); // Keep track of jobs by jobId => job info - private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); public static final long CONSUMPTION_END_TIMEOUT_MS = Duration.ofMinutes(30).toMillis(); - public static final long UPLOAD_END_TIMEOUT_MS = Duration.ofMinutes(5).toMillis(); public static final long CHECK_INTERVAL_MS = Duration.ofSeconds(5).toMillis(); @Inject @@ -121,13 +120,13 @@ public class ReIngestionResource { /** * Simple data class to hold job details. */ - private static class ReIngestionJob { + private static class ReingestionJob { private final String _jobId; private final String _tableNameWithType; private final String _segmentName; private final long _startTimeMs; - ReIngestionJob(String jobId, String tableNameWithType, String segmentName) { + ReingestionJob(String jobId, String tableNameWithType, String segmentName) { _jobId = jobId; _tableNameWithType = tableNameWithType; _segmentName = segmentName; @@ -160,7 +159,7 @@ public long getStartTimeMs() { @ApiOperation("Get all running re-ingestion jobs along with job IDs") public Response getAllRunningReingestionJobs() { // Filter only the jobs still marked as running - List runningJobs = new ArrayList<>(RUNNING_JOBS.values()); + List runningJobs = new ArrayList<>(RUNNING_JOBS.values()); return Response.ok(runningJobs).build(); } @@ -171,10 +170,10 @@ public Response getAllRunningReingestionJobs() { @ApiOperation(value = "Re-ingest segment asynchronously", notes = "Returns a jobId immediately; ingestion runs in " + "background.") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), + @ApiResponse(code = 200, message = "Success", response = ReingestionResponse.class), @ApiResponse(code = 500, message = "Internal server error", response = ErrorInfo.class) }) - public Response reIngestSegment(@PathParam("segmentName") String segmentName) { + public Response reingestSegment(@PathParam("segmentName") String segmentName) { // if segment is not in LLC format, return error if (!LLCSegmentName.isLLCSegment(segmentName)) { throw new WebApplicationException("Segment name is not in LLC format: " + segmentName, @@ -241,7 +240,7 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { // Generate a jobId for tracking String jobId = UUID.randomUUID().toString(); - ReIngestionJob job = new ReIngestionJob(jobId, tableNameWithType, segmentName); + ReingestionJob job = new ReingestionJob(jobId, tableNameWithType, segmentName); // Kick off the actual work asynchronously REINGESTION_EXECUTOR.submit(() -> { @@ -257,7 +256,7 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { tableDataManager.getSegmentBuildSemaphore(), null); RUNNING_JOBS.put(jobId, job); - doReIngestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); + doReingestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); } catch (Exception e) { LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); } finally { @@ -266,16 +265,16 @@ public Response reIngestSegment(@PathParam("segmentName") String segmentName) { } }); - ReIngestionResponse immediateResponse = new ReIngestionResponse( + ReingestionResponse immediateResponse = new ReingestionResponse( "Re-ingestion job submitted successfully with jobId: " + jobId); return Response.ok(immediateResponse).build(); } /** * The actual re-ingestion logic, moved into a separate method for clarity. - * This is essentially the old synchronous logic you had in reIngestSegment. + * This is essentially the old synchronous logic you had in reingestSegment. */ - private void doReIngestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmentName llcSegmentName, + private void doReingestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmentName llcSegmentName, String tableNameWithType, IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) throws Exception { try { diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionRequest.java similarity index 97% rename from pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java rename to pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionRequest.java index a2b92379e549..06965db15ac1 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionRequest.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionRequest.java @@ -18,7 +18,7 @@ */ package org.apache.pinot.server.api.resources.reingestion; -public class ReIngestionRequest { +public class ReingestionRequest { private String _tableNameWithType; private String _segmentName; diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionResponse.java similarity index 92% rename from pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java rename to pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionResponse.java index af1b5f7d55bb..63b7269d937a 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReIngestionResponse.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/ReingestionResponse.java @@ -18,10 +18,10 @@ */ package org.apache.pinot.server.api.resources.reingestion; -public class ReIngestionResponse { +public class ReingestionResponse { private String _message; - public ReIngestionResponse(String message) { + public ReingestionResponse(String message) { _message = message; } From 837aa260487a45c1fbce3bbcf2307643c3a14567 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 08:59:16 +0530 Subject: [PATCH 59/65] Replace reIngest with reingest --- .../org/apache/pinot/common/utils/FileUploadDownloadClient.java | 2 +- .../resources/PinotSegmentUploadDownloadRestletResource.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java index 162af381225e..6c9787222aee 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java @@ -127,7 +127,7 @@ public static FileUploadType getDefaultUploadType() { private static final String FORCE_CLEANUP_PARAMETER = "&forceCleanup="; private static final String RETENTION_PARAMETER = "retention="; - public static final String SEGMENT_REINGEST_COMPLETION_PATH = "/segment/reingested"; + public static final String SEGMENT_REINGEST_COMPLETION_PATH = "/segments/reingested"; private static final List SUPPORTED_PROTOCOLS = Arrays.asList(HTTP, HTTPS); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index 92125555ba3b..f8360641036e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -976,7 +976,7 @@ public Response revertReplaceSegments( @ManagedAsync @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.MULTIPART_FORM_DATA) - @Path("segment/reingested") + @Path("segments/reingested") @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.UPLOAD_SEGMENT) @Authenticate(AccessType.CREATE) @ApiOperation(value = "Reingest a realtime segment", notes = "Reingest a segment as multipart file") From 09e8583109c568a08120a59bce281d21bf2e6ab8 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 09:07:45 +0530 Subject: [PATCH 60/65] Ensure correctness in reingestion --- .../api/resources/ReingestionResource.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java index c6e80c50666b..511e44afa655 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java @@ -41,9 +41,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; import javax.inject.Inject; @@ -96,18 +94,17 @@ public class ReingestionResource { private static final Logger LOGGER = LoggerFactory.getLogger(ReingestionResource.class); //TODO: Make this configurable - private static final int MIN_REINGESTION_THREADS = 4; - private static final int MAX_PARALLEL_REINGESTIONS = 8; + private static final int MIN_REINGESTION_THREADS = 2; + private static final int MAX_PARALLEL_REINGESTIONS = + Math.max(Runtime.getRuntime().availableProcessors() / 2, MIN_REINGESTION_THREADS); // Tracks if a particular segment is currently being re-ingested private static final ConcurrentHashMap SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); // Executor for asynchronous re-ingestion - private static final ExecutorService REINGESTION_EXECUTOR = - new ThreadPoolExecutor(MIN_REINGESTION_THREADS, MAX_PARALLEL_REINGESTIONS, 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(), // unbounded queue for the reingestion tasks - new ThreadFactoryBuilder().setNameFormat("reingestion-worker-%d").build()); + private static final ExecutorService REINGESTION_EXECUTOR = Executors.newFixedThreadPool(MAX_PARALLEL_REINGESTIONS, + new ThreadFactoryBuilder().setNameFormat("reingestion-worker-%d").build()); // Keep track of jobs by jobId => job info private static final ConcurrentHashMap RUNNING_JOBS = new ConcurrentHashMap<>(); @@ -262,6 +259,7 @@ public Response reingestSegment(@PathParam("segmentName") String segmentName) { } finally { isIngesting.set(false); RUNNING_JOBS.remove(jobId); + SEGMENT_INGESTION_MAP.remove(segmentName); } }); From 3d6fdf49f97719a645c252ae09ab306ad1aa7556 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 09:34:50 +0530 Subject: [PATCH 61/65] Decouple reingest completion and upload ZK path --- ...tSegmentUploadDownloadRestletResource.java | 15 ++-- .../controller/api/upload/ZKOperator.java | 72 ++++++++++--------- 2 files changed, 48 insertions(+), 39 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index f8360641036e..96cd70ec30ae 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -239,7 +239,7 @@ private SuccessResponse uploadSegment(@Nullable String tableName, TableType tabl private SuccessResponse uploadSegment(@Nullable String tableName, TableType tableType, @Nullable FormDataMultiPart multiPart, boolean copySegmentToFinalLocation, boolean enableParallelPushProtection, - boolean allowRefresh, boolean allowReset, HttpHeaders headers, Request request) { + boolean allowRefresh, boolean isReingested, HttpHeaders headers, Request request) { if (StringUtils.isNotEmpty(tableName)) { TableType tableTypeFromTableName = TableNameBuilder.getTableTypeFromTableName(tableName); if (tableTypeFromTableName != null && tableTypeFromTableName != tableType) { @@ -416,9 +416,16 @@ private SuccessResponse uploadSegment(@Nullable String tableName, TableType tabl segmentDownloadURIStr, segmentFile, tableNameWithType, copySegmentToFinalLocation); ZKOperator zkOperator = new ZKOperator(_pinotHelixResourceManager, _controllerConf, _controllerMetrics); - zkOperator.completeSegmentOperations(tableNameWithType, segmentMetadata, uploadType, finalSegmentLocationURI, - segmentFile, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, - enableParallelPushProtection, allowRefresh, allowReset, headers); + + if (!isReingested) { + zkOperator.completeSegmentOperations(tableNameWithType, segmentMetadata, uploadType, finalSegmentLocationURI, + segmentFile, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, + enableParallelPushProtection, allowRefresh, headers); + } else { + zkOperator.completeReingestedSegmentOperations(tableNameWithType, segmentMetadata, finalSegmentLocationURI, + sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, + headers); + } return new SuccessResponse("Successfully uploaded segment: " + segmentName + " of table: " + tableNameWithType); } catch (WebApplicationException e) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java index a10dd8075502..2b6e8717a8a0 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/ZKOperator.java @@ -72,17 +72,6 @@ public void completeSegmentOperations(String tableNameWithType, SegmentMetadata @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, boolean allowRefresh, HttpHeaders headers) throws Exception { - completeSegmentOperations(tableNameWithType, segmentMetadata, uploadType, finalSegmentLocationURI, segmentFile, - sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, - allowRefresh, false, headers); - } - - public void completeSegmentOperations(String tableNameWithType, SegmentMetadata segmentMetadata, - FileUploadType uploadType, @Nullable URI finalSegmentLocationURI, File segmentFile, - @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, - long segmentSizeInBytes, boolean enableParallelPushProtection, boolean allowRefresh, boolean allowReset, - HttpHeaders headers) - throws Exception { String segmentName = segmentMetadata.getName(); boolean refreshOnly = Boolean.parseBoolean(headers.getHeaderString(FileUploadDownloadClient.CustomHeaders.REFRESH_ONLY)); @@ -112,7 +101,7 @@ public void completeSegmentOperations(String tableNameWithType, SegmentMetadata } else { // Refresh an existing segment - if (!(allowRefresh || allowReset)) { + if (!(allowRefresh)) { // We cannot perform this check up-front in UploadSegment API call. If a segment doesn't exist during the check // done up-front but ends up getting created before the check here, we could incorrectly refresh an existing // segment. @@ -121,17 +110,10 @@ public void completeSegmentOperations(String tableNameWithType, SegmentMetadata segmentName, tableNameWithType), Response.Status.CONFLICT); } - if (allowRefresh) { - LOGGER.info("Segment: {} already exists in table: {}, refreshing it", segmentName, tableNameWithType); - processExistingSegmentWithRefresh(tableNameWithType, segmentMetadata, uploadType, - existingSegmentMetadataZNRecord, finalSegmentLocationURI, segmentFile, sourceDownloadURIStr, - segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, headers); - } else if (allowReset) { - LOGGER.info("Segment: {} already exists in table: {}, resetting it", segmentName, tableNameWithType); - processExistingSegmentWithReset(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, - finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, - enableParallelPushProtection, headers); - } + LOGGER.info("Segment: {} already exists in table: {}, refreshing it", segmentName, tableNameWithType); + processExistingSegmentWithRefresh(tableNameWithType, segmentMetadata, uploadType, + existingSegmentMetadataZNRecord, finalSegmentLocationURI, segmentFile, sourceDownloadURIStr, + segmentDownloadURIStr, crypterName, segmentSizeInBytes, enableParallelPushProtection, headers); } } @@ -190,6 +172,29 @@ public void completeSegmentsOperations(String tableNameWithType, FileUploadType processExistingSegments(tableNameWithType, uploadType, enableParallelPushProtection, headers, existingSegmentsList); } + public void completeReingestedSegmentOperations(String tableNameWithType, SegmentMetadata segmentMetadata, + @Nullable URI finalSegmentLocationURI, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, + @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) + throws Exception { + String segmentName = segmentMetadata.getName(); + + ZNRecord existingSegmentMetadataZNRecord = + _pinotHelixResourceManager.getSegmentMetadataZnRecord(tableNameWithType, segmentName); + if (existingSegmentMetadataZNRecord != null && shouldProcessAsNewSegment(tableNameWithType, segmentName, + existingSegmentMetadataZNRecord, enableParallelPushProtection)) { + LOGGER.warn("Removing segment ZK metadata (recovering from previous upload failure) for table: {}, segment: {}", + tableNameWithType, segmentName); + Preconditions.checkState(_pinotHelixResourceManager.removeSegmentZKMetadata(tableNameWithType, segmentName), + "Failed to remove segment ZK metadata for table: %s, segment: %s", tableNameWithType, segmentName); + existingSegmentMetadataZNRecord = null; + } + + LOGGER.info("Segment: {} already exists in table: {}, resetting it", segmentName, tableNameWithType); + processReingestedSegment(tableNameWithType, segmentMetadata, existingSegmentMetadataZNRecord, + finalSegmentLocationURI, sourceDownloadURIStr, segmentDownloadURIStr, crypterName, segmentSizeInBytes, + enableParallelPushProtection, headers); + } + /** * Returns {@code true} when the segment should be processed as new segment. *

When segment ZK metadata exists, check if segment exists in the ideal state. If the previous upload failed after @@ -494,7 +499,7 @@ private void processExistingSegments(String tableNameWithType, FileUploadType up } } - private void processExistingSegmentWithReset(String tableNameWithType, SegmentMetadata segmentMetadata, + private void processReingestedSegment(String tableNameWithType, SegmentMetadata segmentMetadata, ZNRecord existingSegmentMetadataZNRecord, @Nullable URI finalSegmentLocationURI, @Nullable String sourceDownloadURIStr, String segmentDownloadURIStr, @Nullable String crypterName, long segmentSizeInBytes, boolean enableParallelPushProtection, HttpHeaders headers) @@ -533,20 +538,17 @@ private void processExistingSegmentWithReset(String tableNameWithType, SegmentMe segmentZKMetadata.setSegmentUploadStartTime(-1); try { - // Update ZK metadata and refresh the segment if necessary - long newCrc = Long.parseLong(segmentMetadata.getCrc()); - // New segment is different with the existing one, update ZK metadata and refresh the segment - LOGGER.info( - "New segment crc {} is different than the existing segment crc {}. Updating ZK metadata and refreshing " - + "segment {}", newCrc, existingCrc, segmentName); - if (finalSegmentLocationURI != null) { - copyFromSegmentURIToDeepStore(new URI(sourceDownloadURIStr), finalSegmentLocationURI); - LOGGER.info("Copied segment: {} of table: {} to final location: {}", segmentName, tableNameWithType, - finalSegmentLocationURI); - } + // Update ZK metadata + if (finalSegmentLocationURI != null) { + copyFromSegmentURIToDeepStore(new URI(sourceDownloadURIStr), finalSegmentLocationURI); + LOGGER.info("Copied segment: {} of table: {} to final location: {}", segmentName, tableNameWithType, + finalSegmentLocationURI); + } // If no modifier is provided, use the custom map from the segment metadata segmentZKMetadata.setCustomMap(segmentMetadata.getCustomMap()); + + // using committing segment zk metadata so that Status is shown as DONE instead of UPLOADED SegmentZKMetadataUtils.updateCommittingSegmentZKMetadata(tableNameWithType, segmentZKMetadata, segmentMetadata, segmentDownloadURIStr, segmentSizeInBytes, segmentZKMetadata.getEndOffset()); if (!_pinotHelixResourceManager.updateZkMetadata(tableNameWithType, segmentZKMetadata, expectedVersion)) { From b525c7e14244919330402c610df36b3b7bac85dc Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 09:49:24 +0530 Subject: [PATCH 62/65] Refactor Stateless segment writer to have least number of arguments --- .../StatelessRealtimeSegmentWriter.java | 58 +++++++++++-------- .../api/resources/ReingestionResource.java | 12 +--- 2 files changed, 35 insertions(+), 35 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index fd531e50cc72..bb07bbbded00 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -19,6 +19,7 @@ package org.apache.pinot.segment.local.realtime.writer; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import java.io.File; import java.io.IOException; import java.nio.file.Path; @@ -36,7 +37,7 @@ import javax.annotation.Nullable; import org.apache.commons.io.FileUtils; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager; @@ -54,6 +55,7 @@ import org.apache.pinot.spi.config.table.SegmentPartitionConfig; import org.apache.pinot.spi.config.table.SegmentZKPropsConfig; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.plugin.PluginManager; @@ -71,6 +73,8 @@ import org.apache.pinot.spi.stream.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.spi.utils.retry.RetryPolicies; import org.apache.pinot.spi.utils.retry.RetryPolicy; import org.slf4j.Logger; @@ -113,29 +117,32 @@ public class StatelessRealtimeSegmentWriter { private final TransformPipeline _transformPipeline; private volatile boolean _isSuccess = false; private volatile Throwable _consumptionException; - private final ServerMetrics _serverMetrics; - public StatelessRealtimeSegmentWriter(String segmentName, String tableNameWithType, int partitionGroupId, - SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema schema, - IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String startOffsetStr, String endOffsetStr, - Semaphore segBuildSemaphore, ServerMetrics serverMetrics) + public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, IndexLoadingConfig indexLoadingConfig, + @Nullable Semaphore segBuildSemaphore) throws Exception { + Preconditions.checkNotNull(indexLoadingConfig.getTableConfig(), "Table config must be set in index loading config"); + Preconditions.checkNotNull(indexLoadingConfig.getSchema(), "Schema must be set in index loading config"); + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentZKMetadata.getSegmentName()); + + _segmentName = segmentZKMetadata.getSegmentName(); + _partitionGroupId = llcSegmentName.getPartitionGroupId(); _segBuildSemaphore = segBuildSemaphore; - _segmentName = segmentName; - _tableNameWithType = tableNameWithType; - _partitionGroupId = partitionGroupId; + _tableNameWithType = TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(llcSegmentName.getTableName()); _segmentZKMetadata = segmentZKMetadata; - _tableConfig = tableConfig; - _schema = schema; - _streamConfig = streamConfig; + _tableConfig = indexLoadingConfig.getTableConfig(); + _schema = indexLoadingConfig.getSchema(); _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); - _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis());; - _serverMetrics = serverMetrics; + _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); + ; _logger = LoggerFactory.getLogger(StatelessRealtimeSegmentWriter.class.getName() + "_" + _segmentName); + Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(_tableConfig).get(0); + _streamConfig = new StreamConfig(_tableNameWithType, streamConfigMap); + _offsetFactory = StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory(); - _startOffset = _offsetFactory.create(startOffsetStr); - _endOffset = _offsetFactory.create(endOffsetStr); + _startOffset = _offsetFactory.create(segmentZKMetadata.getStartOffset()); + _endOffset = _offsetFactory.create(segmentZKMetadata.getEndOffset()); String clientId = getClientId(); @@ -158,7 +165,7 @@ public StatelessRealtimeSegmentWriter(String segmentName, String tableNameWithTy _decoder = createDecoder(fieldsToRead); // Fetch capacity from indexLoadingConfig or use default - int capacity = streamConfig.getFlushThresholdRows(); + int capacity = _streamConfig.getFlushThresholdRows(); if (capacity <= 0) { capacity = DEFAULT_CAPACITY; } @@ -181,15 +188,15 @@ public StatelessRealtimeSegmentWriter(String segmentName, String tableNameWithTy .setSegmentZKMetadata(_segmentZKMetadata).setStatsHistory(statsHistory).setSchema(_schema) .setCapacity(capacity).setAvgNumMultiValues(avgNumMultiValues) .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) - .setFieldConfigList(tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) + .setFieldConfigList(_tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) .setMemoryManager( - new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, _serverMetrics)); + new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, null)); setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); - _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), _serverMetrics); + _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), null); - _transformPipeline = new TransformPipeline(tableConfig, schema); + _transformPipeline = new TransformPipeline(_tableConfig, _schema); // Initialize fetch timeout _fetchTimeoutMs = @@ -338,7 +345,8 @@ public Throwable getConsumptionException() { } @VisibleForTesting - public SegmentBuildDescriptor buildSegmentInternal() throws Exception { + public SegmentBuildDescriptor buildSegmentInternal() + throws Exception { _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); final long startTimeMillis = now(); try { @@ -378,11 +386,11 @@ public SegmentBuildDescriptor buildSegmentInternal() throws Exception { // Build the segment RealtimeSegmentConverter converter = - new RealtimeSegmentConverter(_realtimeSegment, segmentZKPropsConfig, tempSegmentFolder.toString(), - _schema, _tableNameWithType, _tableConfig, _segmentZKMetadata.getSegmentName(), + new RealtimeSegmentConverter(_realtimeSegment, segmentZKPropsConfig, tempSegmentFolder.toString(), _schema, + _tableNameWithType, _tableConfig, _segmentZKMetadata.getSegmentName(), _tableConfig.getIndexingConfig().isNullHandlingEnabled()); try { - converter.build(null, _serverMetrics); + converter.build(null, null); } catch (Exception e) { _logger.error("Failed to build segment", e); FileUtils.deleteQuietly(tempSegmentFolder.toFile()); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java index 511e44afa655..b6ee6d38a056 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java @@ -69,8 +69,6 @@ import org.apache.pinot.server.starter.ServerInstance; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; @@ -242,15 +240,9 @@ public Response reingestSegment(@PathParam("segmentName") String segmentName) { // Kick off the actual work asynchronously REINGESTION_EXECUTOR.submit(() -> { try { - int partitionGroupId = llcSegmentName.getPartitionGroupId(); - - Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - StatelessRealtimeSegmentWriter manager = - new StatelessRealtimeSegmentWriter(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, - tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, - tableDataManager.getSegmentBuildSemaphore(), null); + new StatelessRealtimeSegmentWriter(segmentZKMetadata, indexLoadingConfig, + tableDataManager.getSegmentBuildSemaphore()); RUNNING_JOBS.put(jobId, job); doReingestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); From f7ae25f3f63d43b0549d4e74d8dc3305fca903f1 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 14:10:55 +0530 Subject: [PATCH 63/65] Fix segment reset for pauseless tables --- .../PinotLLCRealtimeSegmentManager.java | 34 ++++++++++++------- .../RealtimeSegmentValidationManager.java | 10 +++--- 2 files changed, 27 insertions(+), 17 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index 9f6c2298b141..9f91dd9b08ec 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -2112,9 +2112,10 @@ URI createSegmentPath(String rawTableName, String segmentName) { * POST http://[serverURL]/reingestSegment/[segmentName] * Request body (JSON): * + * If segment is in ERROR state in only few replicas but has download URL, we instead trigger a segment reset * @param realtimeTableName The table name with type, e.g. "myTable_REALTIME" */ - public void reingestSegmentsWithErrorState(String realtimeTableName) { + public void repairSegmentsInErrorState(String realtimeTableName) { // Step 1: Fetch the ExternalView and all segments ExternalView externalView = _helixResourceManager.getTableExternalView(realtimeTableName); if (externalView == null) { @@ -2126,7 +2127,8 @@ public void reingestSegmentsWithErrorState(String realtimeTableName) { Map> segmentToInstanceIdealStateMap = idealState.getRecord().getMapFields(); // find segments in ERROR state in externalView - List segmentsInErrorState = new ArrayList<>(); + List segmentsInErrorStateInAllReplicas = new ArrayList<>(); + List segmentsInErrorStateInAtleastOneReplica = new ArrayList<>(); for (Map.Entry> entry : segmentToInstanceCurrentStateMap.entrySet()) { String segmentName = entry.getKey(); @@ -2137,25 +2139,29 @@ public void reingestSegmentsWithErrorState(String realtimeTableName) { } Map instanceStateMap = entry.getValue(); - boolean allReplicasInError = true; + int numReplicasInError = 0; for (String state : instanceStateMap.values()) { - if (!SegmentStateModel.ERROR.equals(state)) { - allReplicasInError = false; - break; + if (SegmentStateModel.ERROR.equals(state)) { + numReplicasInError++; } } - if (allReplicasInError) { - segmentsInErrorState.add(segmentName); + + if (numReplicasInError > 0) { + segmentsInErrorStateInAtleastOneReplica.add(segmentName); + } + + if (numReplicasInError == instanceStateMap.size()) { + segmentsInErrorStateInAllReplicas.add(segmentName); } } - if (segmentsInErrorState.isEmpty()) { + if (segmentsInErrorStateInAtleastOneReplica.isEmpty()) { LOGGER.info("No segments found in ERROR state for table {}", realtimeTableName); return; } // filter out segments that are not ONLINE in IdealState - for (String segmentName : segmentsInErrorState) { + for (String segmentName : segmentsInErrorStateInAtleastOneReplica) { Map instanceIdealStateMap = segmentToInstanceIdealStateMap.get(segmentName); boolean isOnline = true; for (String state : instanceIdealStateMap.values()) { @@ -2169,8 +2175,9 @@ public void reingestSegmentsWithErrorState(String realtimeTableName) { } SegmentZKMetadata segmentZKMetadata = getSegmentZKMetadata(realtimeTableName, segmentName); - // We only consider segments that are in COMMITTING state - if (segmentZKMetadata.getStatus() == Status.COMMITTING) { + // We only consider segments that are in COMMITTING state for reingestion + if (segmentsInErrorStateInAllReplicas.contains(segmentName) + && segmentZKMetadata.getStatus() == Status.COMMITTING) { Map instanceStateMap = segmentToInstanceIdealStateMap.get(segmentName); // Step 3: “No peer has that segment.” => Re-ingest from one server that is supposed to host it and is alive @@ -2191,6 +2198,9 @@ public void reingestSegmentsWithErrorState(String realtimeTableName) { } catch (Exception e) { LOGGER.error("Failed to call reingestSegment for segment {} on server {}", segmentName, aliveServer, e); } + } else if (segmentZKMetadata.getStatus() != Status.IN_PROGRESS) { + // trigger segment reset for this since it is not in IN_PROGRESS state so download URL must be present + _helixResourceManager.resetSegment(realtimeTableName, segmentName, null); } } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java index 09212c037022..7a9596b938a6 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java @@ -124,7 +124,11 @@ protected void processTable(String tableNameWithType, Context context) { boolean isPauselessConsumptionEnabled = PauselessConsumptionUtils.isPauselessEnabled(tableConfig); if (isPauselessConsumptionEnabled) { - _llcRealtimeSegmentManager.reingestSegmentsWithErrorState(tableConfig.getTableName()); + _llcRealtimeSegmentManager.repairSegmentsInErrorState(tableConfig.getTableName()); + } else if (_segmentAutoResetOnErrorAtValidation) { + // reset for pauseless tables is already handled in repairSegmentsInErrorState method + // with additional checks for pauseless consumption + _pinotHelixResourceManager.resetSegments(tableConfig.getTableName(), null, true); } } @@ -186,10 +190,6 @@ private void runSegmentLevelValidation(TableConfig tableConfig) { if (_llcRealtimeSegmentManager.isDeepStoreLLCSegmentUploadRetryEnabled()) { _llcRealtimeSegmentManager.uploadToDeepStoreIfMissing(tableConfig, segmentsZKMetadata); } - - if (_segmentAutoResetOnErrorAtValidation) { - _pinotHelixResourceManager.resetSegments(realtimeTableName, null, true); - } } @Override From 3c179578a74f1297b071786d603a5ba87c860803 Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 15:11:32 +0530 Subject: [PATCH 64/65] Cleaning up Stateless segment writer --- .../StatelessRealtimeSegmentWriter.java | 117 ++++-------------- .../api/resources/ReingestionResource.java | 45 +------ 2 files changed, 27 insertions(+), 135 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index bb07bbbded00..20895e45a123 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -88,12 +88,14 @@ public class StatelessRealtimeSegmentWriter { private static final int DEFAULT_CAPACITY = 100_000; private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000; + public static final String SEGMENT_STATS_FILE_NAME = "segment-stats.ser"; + public static final String RESOURCE_TMP_DIR_PREFIX = "resourceTmpDir_"; + public static final String RESOURCE_DATA_DIR_PREFIX = "resourceDataDir_"; private final Semaphore _segBuildSemaphore; private final String _segmentName; private final String _tableNameWithType; private final int _partitionGroupId; - private final String _segmentNameStr; private final SegmentZKMetadata _segmentZKMetadata; private final TableConfig _tableConfig; private final Schema _schema; @@ -108,7 +110,6 @@ public class StatelessRealtimeSegmentWriter { private final File _resourceDataDir; private final Logger _logger; private Thread _consumerThread; - private final AtomicBoolean _shouldStop = new AtomicBoolean(false); private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false); private final StreamPartitionMsgOffset _startOffset; private final StreamPartitionMsgOffset _endOffset; @@ -132,9 +133,14 @@ public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, Index _segmentZKMetadata = segmentZKMetadata; _tableConfig = indexLoadingConfig.getTableConfig(); _schema = indexLoadingConfig.getSchema(); - _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); - _resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); - ; + String tableDataDir = indexLoadingConfig.getTableDataDir() == null ? FileUtils.getTempDirectory().getAbsolutePath() + : indexLoadingConfig.getTableDataDir(); + File reingestionDir = new File(tableDataDir, "reingestion"); + _resourceTmpDir = + new File(reingestionDir, RESOURCE_TMP_DIR_PREFIX + _segmentName + "_" + System.currentTimeMillis()); + _resourceDataDir = + new File(reingestionDir, RESOURCE_DATA_DIR_PREFIX + _segmentName + "_" + System.currentTimeMillis()); + _logger = LoggerFactory.getLogger(StatelessRealtimeSegmentWriter.class.getName() + "_" + _segmentName); Map streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(_tableConfig).get(0); @@ -152,7 +158,6 @@ public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, Index _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig); _partitionMetadataProvider = _consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId); - _segmentNameStr = _segmentZKMetadata.getSegmentName(); // Create a simple PartitionGroupConsumptionStatus PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = @@ -175,10 +180,7 @@ public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, Index // Load stats history, here we are using the same stats while as the RealtimeSegmentDataManager so that we are // much more efficient in allocating buffers. It also works with empty file - String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != null - ? indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + File.separator + _tableNameWithType - : _resourceTmpDir.getAbsolutePath(); - File statsHistoryFile = new File(tableDataDir, "segment-stats.ser"); + File statsHistoryFile = new File(tableDataDir, SEGMENT_STATS_FILE_NAME); RealtimeSegmentStatsHistory statsHistory = RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile); // Initialize mutable segment with configurations @@ -190,7 +192,7 @@ public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, Index .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation()) .setFieldConfigList(_tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath()) .setMemoryManager( - new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentNameStr, null)); + new MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), _segmentName, null)); setPartitionParameters(realtimeSegmentConfigBuilder, _tableConfig.getIndexingConfig().getSegmentPartitionConfig()); @@ -256,20 +258,16 @@ public void run() { _logger.info("Created new consumer thread {} for {}", _consumerThread, this); _currentOffset = _startOffset; TransformPipeline.Result reusedResult = new TransformPipeline.Result(); - while (!_shouldStop.get() && _currentOffset.compareTo(_endOffset) < 0) { + while (_currentOffset.compareTo(_endOffset) < 0) { // Fetch messages MessageBatch messageBatch = _consumer.fetchMessages(_currentOffset, _fetchTimeoutMs); int messageCount = messageBatch.getMessageCount(); for (int i = 0; i < messageCount; i++) { - if (_shouldStop.get()) { - break; - } StreamMessage streamMessage = messageBatch.getStreamMessage(i); if (streamMessage.getMetadata() != null && streamMessage.getMetadata().getOffset() != null && streamMessage.getMetadata().getOffset().compareTo(_endOffset) >= 0) { - _shouldStop.set(true); _logger.info("Reached end offset: {} for partition group: {}", _endOffset, _partitionGroupId); break; } @@ -311,7 +309,6 @@ public void run() { } public void stopConsumption() { - _shouldStop.set(true); if (_consumerThread.isAlive()) { _consumerThread.interrupt(); try { @@ -322,16 +319,13 @@ public void stopConsumption() { } } - public void destroy() { + public void close() { + stopConsumption(); _realtimeSegment.destroy(); FileUtils.deleteQuietly(_resourceTmpDir); FileUtils.deleteQuietly(_resourceDataDir); } - public void offload() { - stopConsumption(); - } - public boolean isDoneConsuming() { return _isDoneConsuming.get(); } @@ -345,7 +339,7 @@ public Throwable getConsumptionException() { } @VisibleForTesting - public SegmentBuildDescriptor buildSegmentInternal() + public File buildSegmentInternal() throws Exception { _logger.info("Building segment from {} to {}", _startOffset, _currentOffset); final long startTimeMillis = now(); @@ -359,7 +353,6 @@ public SegmentBuildDescriptor buildSegmentInternal() Duration.between(acquireStart, Instant.now())); timeoutSeconds = Math.min(timeoutSeconds * 2, 300); } - _logger.info("Acquired semaphore for building segment"); } } catch (InterruptedException e) { String errorMessage = "Interrupted while waiting for semaphore"; @@ -369,12 +362,13 @@ public SegmentBuildDescriptor buildSegmentInternal() final long lockAcquireTimeMillis = now(); final long waitTimeMillis = lockAcquireTimeMillis - startTimeMillis; + _logger.info("Acquired lock for building segment in {} ms", waitTimeMillis); // Build a segment from in-memory rows. // Use a temporary directory Path tempSegmentFolder = null; try { tempSegmentFolder = - java.nio.file.Files.createTempDirectory(_resourceTmpDir.toPath(), "tmp-" + _segmentNameStr + "-"); + java.nio.file.Files.createTempDirectory(_resourceTmpDir.toPath(), "tmp-" + _segmentName + "-"); } catch (IOException e) { _logger.error("Failed to create temporary directory for segment build", e); return null; @@ -401,10 +395,10 @@ public SegmentBuildDescriptor buildSegmentInternal() converter.isColumnMajorEnabled(), buildTimeMillis, waitTimeMillis); File dataDir = _resourceDataDir; - File indexDir = new File(dataDir, _segmentNameStr); + File indexDir = new File(dataDir, _segmentName); FileUtils.deleteQuietly(indexDir); - File tempIndexDir = new File(tempSegmentFolder.toFile(), _segmentNameStr); + File tempIndexDir = new File(tempSegmentFolder.toFile(), _segmentName); if (!tempIndexDir.exists()) { _logger.error("Temp index directory {} does not exist", tempIndexDir); FileUtils.deleteQuietly(tempSegmentFolder.toFile()); @@ -422,7 +416,7 @@ public SegmentBuildDescriptor buildSegmentInternal() SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); long segmentSizeBytes = FileUtils.sizeOfDirectory(indexDir); - File segmentTarFile = new File(dataDir, _segmentNameStr + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); + File segmentTarFile = new File(dataDir, _segmentName + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); try { TarCompressionUtils.createCompressedTarFile(indexDir, segmentTarFile); } catch (IOException e) { @@ -443,20 +437,13 @@ public SegmentBuildDescriptor buildSegmentInternal() Map metadataFiles = new HashMap<>(); metadataFiles.put(V1Constants.MetadataKeys.METADATA_FILE_NAME, metadataFile); metadataFiles.put(V1Constants.SEGMENT_CREATION_META, creationMetaFile); - return new SegmentBuildDescriptor(segmentTarFile, metadataFiles, _currentOffset, buildTimeMillis, buildTimeMillis, - segmentSizeBytes, segmentMetadata); + return segmentTarFile; } protected long now() { return System.currentTimeMillis(); } - public void removeSegmentFile(SegmentBuildDescriptor segmentBuildDescriptor) { - if (segmentBuildDescriptor != null) { - segmentBuildDescriptor.deleteSegmentFile(); - } - } - /* * set the following partition parameters in RT segment config builder: * - partition column @@ -529,62 +516,4 @@ private void closePartitionMetadataProvider() { } } } - - public class SegmentBuildDescriptor { - final File _segmentTarFile; - final Map _metadataFileMap; - final StreamPartitionMsgOffset _offset; - final long _waitTimeMillis; - final long _buildTimeMillis; - final long _segmentSizeBytes; - final SegmentMetadataImpl _segmentMetadata; - - public SegmentBuildDescriptor(@Nullable File segmentTarFile, @Nullable Map metadataFileMap, - StreamPartitionMsgOffset offset, long buildTimeMillis, long waitTimeMillis, long segmentSizeBytes, - SegmentMetadataImpl segmentMetadata) { - _segmentTarFile = segmentTarFile; - _metadataFileMap = metadataFileMap; - _offset = _offsetFactory.create(offset); - _buildTimeMillis = buildTimeMillis; - _waitTimeMillis = waitTimeMillis; - _segmentSizeBytes = segmentSizeBytes; - _segmentMetadata = segmentMetadata; - } - - public StreamPartitionMsgOffset getOffset() { - return _offset; - } - - public long getBuildTimeMillis() { - return _buildTimeMillis; - } - - public long getWaitTimeMillis() { - return _waitTimeMillis; - } - - @Nullable - public File getSegmentTarFile() { - return _segmentTarFile; - } - - @Nullable - public Map getMetadataFiles() { - return _metadataFileMap; - } - - public long getSegmentSizeBytes() { - return _segmentSizeBytes; - } - - public void deleteSegmentFile() { - if (_segmentTarFile != null) { - FileUtils.deleteQuietly(_segmentTarFile); - } - } - - public SegmentMetadataImpl getSegmentMetadata() { - return _segmentMetadata; - } - } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java index b6ee6d38a056..c44a4b462ee4 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReingestionResource.java @@ -29,21 +29,14 @@ import io.swagger.annotations.SecurityDefinition; import io.swagger.annotations.SwaggerDefinition; import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -55,13 +48,10 @@ import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.utils.LLCSegmentName; -import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager; -import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.realtime.writer.StatelessRealtimeSegmentWriter; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.server.api.resources.reingestion.ReingestionResponse; @@ -69,7 +59,6 @@ import org.apache.pinot.server.starter.ServerInstance; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.utils.StringUtil; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -245,7 +234,7 @@ public Response reingestSegment(@PathParam("segmentName") String segmentName) { tableDataManager.getSegmentBuildSemaphore()); RUNNING_JOBS.put(jobId, job); - doReingestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig, tableDataManager); + doReingestSegment(manager, llcSegmentName, tableNameWithType, indexLoadingConfig); } catch (Exception e) { LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); } finally { @@ -265,7 +254,7 @@ public Response reingestSegment(@PathParam("segmentName") String segmentName) { * This is essentially the old synchronous logic you had in reingestSegment. */ private void doReingestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmentName llcSegmentName, - String tableNameWithType, IndexLoadingConfig indexLoadingConfig, TableDataManager tableDataManager) + String tableNameWithType, IndexLoadingConfig indexLoadingConfig) throws Exception { try { String segmentName = llcSegmentName.getSegmentName(); @@ -279,10 +268,7 @@ private void doReingestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmen } LOGGER.info("Starting build for segment {}", segmentName); - StatelessRealtimeSegmentWriter.SegmentBuildDescriptor segmentBuildDescriptor = - manager.buildSegmentInternal(); - - File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + File segmentTarFile = manager.buildSegmentInternal(); if (segmentTarFile == null) { throw new Exception("Failed to build segment: " + segmentName); } @@ -292,8 +278,7 @@ private void doReingestSegment(StatelessRealtimeSegmentWriter manager, LLCSegmen protocolHandler.uploadReingestedSegment(segmentName, indexLoadingConfig.getSegmentStoreURI(), segmentTarFile); LOGGER.info("Re-ingested segment {} uploaded successfully", segmentName); } finally { - manager.offload(); - manager.destroy(); + manager.close(); } } @@ -325,26 +310,4 @@ private void waitForCondition( throw new RuntimeException("Timeout waiting for condition: " + condition); } - - public void resetSegment(HttpClient httpClient, String controllerVipUrl, String tableNameWithType, String segmentName, - String targetInstance, Map headers) - throws IOException { - try { - HttpClient.wrapAndThrowHttpException(httpClient.sendJsonPostRequest( - new URI(getURLForSegmentReset(controllerVipUrl, tableNameWithType, segmentName, targetInstance)), null, - headers)); - } catch (HttpErrorStatusException | URISyntaxException e) { - throw new IOException(e); - } - } - - private String getURLForSegmentReset(String controllerVipUrl, String tableNameWithType, String segmentName, - @Nullable String targetInstance) { - String query = targetInstance == null ? "reset" : "reset?targetInstance=" + targetInstance; - return StringUtil.join("/", controllerVipUrl, "segments", tableNameWithType, encode(segmentName), query); - } - - private String encode(String s) { - return URLEncoder.encode(s, StandardCharsets.UTF_8); - } } From a3fa25c3d1308c35f3170ac78c1df72040bce46d Mon Sep 17 00:00:00 2001 From: KKCorps Date: Thu, 6 Feb 2025 17:35:10 +0530 Subject: [PATCH 65/65] Rename metric --- .../java/org/apache/pinot/common/metrics/ControllerGauge.java | 4 ++-- .../resources/PinotSegmentUploadDownloadRestletResource.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java index dfa1f1e09d95..415acc0596b6 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java @@ -182,8 +182,8 @@ public enum ControllerGauge implements AbstractMetrics.Gauge { TABLE_REBALANCE_IN_PROGRESS("tableRebalanceInProgress", false), - // Number of in progress segment reingestion - SEGMENT_REINGESTION_IN_PROGRESS("segmentReingestionInProgress", true); + // Number of reingested segments getting uploaded + SEGMENT_REINGESTION_UPLOAD_IN_PROGRESS("segmentReingestionUploadInProgress", true); private final String _gaugeName; private final String _unit; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java index 96cd70ec30ae..4b4cf49c3e0e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java @@ -996,7 +996,7 @@ public Response revertReplaceSegments( @ApiResponse(code = 500, message = "Internal error") }) @TrackInflightRequestMetrics - @TrackedByGauge(gauge = ControllerGauge.SEGMENT_REINGESTION_IN_PROGRESS) + @TrackedByGauge(gauge = ControllerGauge.SEGMENT_REINGESTION_UPLOAD_IN_PROGRESS) public void completeSegmentReingestion(FormDataMultiPart multiPart, @ApiParam(value = "Name of the table") @QueryParam(FileUploadDownloadClient.QueryParameters.TABLE_NAME) String tableName,